You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by HeartSaVioR <gi...@git.apache.org> on 2018/08/17 23:20:28 UTC

[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

GitHub user HeartSaVioR opened a pull request:

    https://github.com/apache/spark/pull/22138

    [SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer

    ## What changes were proposed in this pull request?
    
    KafkaDataConsumer contains its own logic for caching InternalKafkaConsumer which looks like can be simplified via applying Apache Commons Pool. Benefits of applying Apache Commons Pool are following:
    
    * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer.
    * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily.
    * We can get various statistics for the object pool, and also be able to enable JMX for the pool.
    
    This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module.
    
    ## How was this patch tested?
    
    Existing unit tests as well as new tests for object pool.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/HeartSaVioR/spark SPARK-25151

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/22138.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #22138
    
----
commit c82f3064fa8744f91b5c8a92645588dc9d53ba35
Author: Jungtaek Lim <ka...@...>
Date:   2018-08-17T09:56:31Z

    [SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer

----


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214853362
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    I know it's old code but but this blocks to solve the multiple consumer extra seek problem.
    If this would be readonly it doesn't matter which task gets which consumer. Consumer just takes a look at the offset indexed data pool for data, gives it back and makes prefetch again.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214908731
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    --- End diff --
    
    This is actually one of optimization kept in existing cache: if `kafkaParams` is cheap enough to be added to CacheKey I would do it. Would we be better to add `require` to do assertion?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214913221
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    +      setMaxTotal(-1)
    +
    +      // Set minimum evictable idle time which will be referred from evictor thread
    +      setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis)
    +      setSoftMinEvictableIdleTimeMillis(-1)
    +
    +      // evictor thread will run test with ten idle objects
    +      setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis)
    +      setNumTestsPerEvictionRun(10)
    +      setEvictionPolicy(new DefaultEvictionPolicy[InternalKafkaConsumer]())
    +
    +      // Immediately fail on exhausted pool while borrowing
    +      setBlockWhenExhausted(false)
    +
    +      setJmxEnabled(jmxEnabled)
    +      setJmxNamePrefix("kafka010-cached-simple-kafka-consumer-pool")
    +    }
    +  }
    +
    +  object PoolConfig {
    +    val CONFIG_NAME_PREFIX = "spark.sql.kafkaConsumerCache."
    +    val CONFIG_NAME_CAPACITY = CONFIG_NAME_PREFIX + "capacity"
    +    val CONFIG_NAME_JMX_ENABLED = CONFIG_NAME_PREFIX + "jmx.enable"
    +    val CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS = CONFIG_NAME_PREFIX +
    +      "minEvictableIdleTimeMillis"
    +    val CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = CONFIG_NAME_PREFIX +
    +      "evictorThreadRunIntervalMillis"
    +
    +    val DEFAULT_VALUE_CAPACITY = 64
    +    val DEFAULT_VALUE_JMX_ENABLED = false
    +    val DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS = 5 * 60 * 1000 // 5 minutes
    +    val DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = 3 * 60 * 1000 // 3 minutes
    +  }
    +
    +  class ObjectFactory extends BaseKeyedPooledObjectFactory[CacheKey, InternalKafkaConsumer]
    +    with Logging {
    +
    +    val keyToKafkaParams: ConcurrentHashMap[CacheKey, ju.Map[String, Object]] =
    +      new ConcurrentHashMap[CacheKey, ju.Map[String, Object]]()
    +
    +    override def create(key: CacheKey): InternalKafkaConsumer = {
    +      val kafkaParams = keyToKafkaParams.get(key)
    +      if (kafkaParams == null) {
    --- End diff --
    
    No strong opinion on this. Looks like Spark allows using `null` and doesn't enforce guarding with `Option` so I just leveraged what `ConcurrentHashMap.get` provides. Did you intend `.getOrElse(throw ...)`?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @zsxwing 
    If it means code freeze for 2.4 is just around the corner then sure! We can focus on blockers for releasing 2.4, and revisit this again. Let me reflect @gaborgsomogyi review comments first (except arguments, so that I can hear more voices on these things).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215270783
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    I know it's the same in the original implementation but this is vulnerable to short time resource allocation flood which can't be handled with idle time.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this please


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    > Do you have more to review?
    
    No. BTW I've just seen a -1 for 2.4 RC...



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95800 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95800/testReport)** for PR 22138 at commit [`107880e`](https://github.com/apache/spark/commit/107880e0e7fb946a0ebf838739320699be968083).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r211053868
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -425,70 +381,36 @@ private[kafka010] object KafkaDataConsumer extends Logging {
       def acquire(
           topicPartition: TopicPartition,
           kafkaParams: ju.Map[String, Object],
    -      useCache: Boolean): KafkaDataConsumer = synchronized {
    -    val key = new CacheKey(topicPartition, kafkaParams)
    -    val existingInternalConsumer = cache.get(key)
    +      useCache: Boolean): KafkaDataConsumer = {
     
    -    lazy val newInternalConsumer = new InternalKafkaConsumer(topicPartition, kafkaParams)
    +    if (!useCache) {
    +      return NonCachedKafkaDataConsumer(new InternalKafkaConsumer(topicPartition, kafkaParams))
    +    }
     
    -    if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) {
    -      // If this is reattempt at running the task, then invalidate cached consumer if any and
    -      // start with a new one.
    -      if (existingInternalConsumer != null) {
    -        // Consumer exists in cache. If its in use, mark it for closing later, or close it now.
    -        if (existingInternalConsumer.inUse) {
    -          existingInternalConsumer.markedForClose = true
    -        } else {
    -          existingInternalConsumer.close()
    -        }
    -      }
    -      cache.remove(key)  // Invalidate the cache in any case
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    val key = new CacheKey(topicPartition, kafkaParams)
     
    -    } else if (!useCache) {
    -      // If planner asks to not reuse consumers, then do not use it, return a new consumer
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumer if any.
     
    -    } else if (existingInternalConsumer == null) {
    -      // If consumer is not already cached, then put a new in the cache and return it
    -      cache.put(key, newInternalConsumer)
    -      newInternalConsumer.inUse = true
    -      CachedKafkaDataConsumer(newInternalConsumer)
    +      // invalidate all idle consumers for the key
    +      pool.invalidateKey(key)
     
    -    } else if (existingInternalConsumer.inUse) {
    -      // If consumer is already cached but is currently in use, then return a new consumer
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +      // borrow a consumer from pool even in this case
    +    }
     
    -    } else {
    -      // If consumer is already cached and is currently not in use, then return that consumer
    -      existingInternalConsumer.inUse = true
    -      CachedKafkaDataConsumer(existingInternalConsumer)
    +    try {
    +      CachedKafkaDataConsumer(pool.borrowObject(key, kafkaParams))
    +    } catch { case _: NoSuchElementException =>
    +      // There's neither idle object to clean up nor available space in pool:
    +      // fail back to create non-cached consumer
    --- End diff --
    
    This approach introduces behavior change: even though `cache` had capacity, the `cache` worked like soft capacity and allowed adding item to the cache when there's neither idle object nor free space. 
    
    New behavior of the KafkaDataConsumer is creating all the objects to non-cached whenever pool is exhausted and there's no idle object to free up.
    
    I think it is not a big deal when we configure "spark.sql.kafkaConsumerCache.capacity" properly, and having hard capacity feels more convenient to determine what's going on.
    
    However we can still mimic the current behavior with having infinite capacity, so we can be back to current behavior if we feel it makes more sense.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95784 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95784/testReport)** for PR 22138 at commit [`2da9924`](https://github.com/apache/spark/commit/2da99247da82036abec7dfbb584f9bb7ff99c577).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r219418553
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    OK.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214907878
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    This is intended to be unbounded, because existing behavior doesn't block borrowing consumers. It just used non-cached consumers instead, and it always succeeded so we should ensure this behavior too. 
    IMHO min idle/max idle is the key to maintain optimal pool, so it may be good for expert to enable configuration on these values for tuning pool.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214910482
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    --- End diff --
    
    Will address.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #98909 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98909/testReport)** for PR 22138 at commit [`fd4ff83`](https://github.com/apache/spark/commit/fd4ff833b6c2b5889d55ee4053970b56ee2b273d).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214916493
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    Allowing fields to be mutable is the optimization for avoiding new instance for every batch. FetchedDataPool ensures each FetchedData doesn't be provided to multiple tasks concurrently, and _nextOffsetInFetchedData is updated once it is returning to pool so that it can be offset information for searching.
    
    Please let me know if I'm missing something here.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214916741
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    +    private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +    private var _nextOffsetInFetchedData: Long,
    +    private var _offsetAfterPoll: Long) {
    +
    +  def withNewPoll(
    +      records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +      offsetAfterPoll: Long): FetchedData = {
    +    this._records = records
    +    this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    +    this._offsetAfterPoll = offsetAfterPoll
    +    this
    +  }
     
    -    /** Move `records` backward and return the previous record. */
    -    def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      assert(_records.hasPrevious, "fetchedData cannot move back")
    -      val record = _records.previous()
    -      _nextOffsetInFetchedData = record.offset
    -      record
    -    }
    +  /** Whether there are more elements */
    +  def hasNext: Boolean = _records.hasNext
     
    -    /** Reset the internal pre-fetched data. */
    -    def reset(): Unit = {
    -      _records = ju.Collections.emptyListIterator()
    -    }
    +  /** Move `records` forward and return the next record. */
    +  def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    val record = _records.next()
    +    _nextOffsetInFetchedData = record.offset + 1
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset in `records`. We use this to verify if we should check if the
    -     * pre-fetched data is still valid.
    -     */
    -    def nextOffsetInFetchedData: Long = _nextOffsetInFetchedData
    +  /** Move `records` backward and return the previous record. */
    +  def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    assert(_records.hasPrevious, "fetchedData cannot move back")
    +    val record = _records.previous()
    +    _nextOffsetInFetchedData = record.offset
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset to poll after draining the pre-fetched records.
    -     */
    -    def offsetAfterPoll: Long = _offsetAfterPoll
    +  /** Reset the internal pre-fetched data. */
    +  def reset(): Unit = {
    +    _records = ju.Collections.emptyListIterator()
       }
     
       /**
    -   * The internal object returned by the `fetchRecord` method. If `record` is empty, it means it is
    -   * invisible (either a transaction message, or an aborted message when the consumer's
    -   * `isolation.level` is `read_committed`), and the caller should use `nextOffsetToFetch` to fetch
    -   * instead.
    +   * Returns the next offset in `records`. We use this to verify if we should check if the
    +   * pre-fetched data is still valid.
        */
    -  private case class FetchedRecord(
    -      var record: ConsumerRecord[Array[Byte], Array[Byte]],
    -      var nextOffsetToFetch: Long) {
    -
    -    def withRecord(
    -        record: ConsumerRecord[Array[Byte], Array[Byte]],
    -        nextOffsetToFetch: Long): FetchedRecord = {
    -      this.record = record
    -      this.nextOffsetToFetch = nextOffsetToFetch
    -      this
    -    }
    -  }
    +  def nextOffsetInFetchedData: Long = _nextOffsetInFetchedData
     
    -  private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +  /**
    +   * Returns the next offset to poll after draining the pre-fetched records.
    +   */
    +  def offsetAfterPoll: Long = _offsetAfterPoll
    +}
     
    -  @volatile private var consumer = createConsumer
    +/**
    + * The internal object returned by the `fetchRecord` method. If `record` is empty, it means it is
    + * invisible (either a transaction message, or an aborted message when the consumer's
    + * `isolation.level` is `read_committed`), and the caller should use `nextOffsetToFetch` to fetch
    + * instead.
    + */
    +private[kafka010] case class FetchedRecord(
    +    var record: ConsumerRecord[Array[Byte], Array[Byte]],
    +    var nextOffsetToFetch: Long) {
    +
    +  def withRecord(
    +      record: ConsumerRecord[Array[Byte], Array[Byte]],
    +      nextOffsetToFetch: Long): FetchedRecord = {
    +    this.record = record
    +    this.nextOffsetToFetch = nextOffsetToFetch
    +    this
    +  }
    +}
     
    -  /** indicates whether this consumer is in use or not */
    -  @volatile var inUse = true
    +/**
    + * This class helps caller to read from Kafka leveraging consumer pool as well as fetched data pool.
    + * This class throws error when data loss is detected while reading from Kafka.
    + *
    + * NOTE for contributors: we need to ensure all the public methods to initialize necessary resources
    + * via calling `ensureConsumerAvailable` and `ensureFetchedDataAvailable`.
    + */
    +private[kafka010] class KafkaDataConsumer(
    +    topicPartition: TopicPartition,
    +    kafkaParams: ju.Map[String, Object],
    +    consumerPool: InternalKafkaConsumerPool,
    +    fetchedDataCache: FetchedDataPool) extends Logging {
    --- End diff --
    
    Nice catch. I was named it to cache and renamed it to pool. Will rename the value name to `fetchedDataPool`.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95502/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96337/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96250 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96250/testReport)** for PR 22138 at commit [`d2f07b7`](https://github.com/apache/spark/commit/d2f07b7796f281c6e020a0f2dcabb9f4a7c37a75).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95472 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95472/testReport)** for PR 22138 at commit [`648550a`](https://github.com/apache/spark/commit/648550ad7028b5576457caf8bbcac0e415366674).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215274150
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -414,17 +468,37 @@ private[kafka010] case class InternalKafkaConsumer(
         }
       }
     
    -  /** Create a new consumer and reset cached states */
    -  private def resetConsumer(): Unit = {
    -    consumer.close()
    -    consumer = createConsumer
    -    fetchedData.reset()
    +  /**
    +   * Poll messages from Kafka starting from `offset` and update `fetchedData`. `fetchedData` may be
    +   * empty if the Kafka consumer fetches some messages but all of them are not visible messages
    +   * (either transaction messages, or aborted messages when `isolation.level` is `read_committed`).
    +   *
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
    +   */
    +  private def fetchData(offset: Long, pollTimeoutMs: Long): Unit = {
    +    val (records, offsetAfterPoll) = consumer.fetch(offset, pollTimeoutMs)
    +    fetchedData.withNewPoll(records.listIterator, offsetAfterPoll)
    +  }
    +
    +  private def ensureConsumerAvailable(): Unit = {
    +    if (consumer == null) {
    --- End diff --
    
    Same here.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218709909
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,243 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    val oldKafkaParams = objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +    require(oldKafkaParams == null || kafkaParams == oldKafkaParams, "Kafka parameters for same " +
    +      s"cache key should be equal. old parameters: $oldKafkaParams new parameters: $kafkaParams")
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    --- End diff --
    
    I don't see where it's used.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    I just applied a new approach: "separate of concerns". This approach does pooling for both kafka consumers as well as fetched data.
    
    Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks.
    
    It also enables applying different policies on pool, which helps optimization of pooling for each pool.
    
    We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This approach properly handles the case.
    
    It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter.
    
    @koeninger @tdas @zsxwing @arunmahadevan 
    Could you please take a look at the new approach? I think this approach solves multiple issues existing code has.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215275456
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    --- End diff --
    
    I meant some kind of assertion. When all goes well it should never fire but if somebody touches this codepart later and doesn't read this, can be notified easily.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214815260
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    +    private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +    private var _nextOffsetInFetchedData: Long,
    +    private var _offsetAfterPoll: Long) {
    +
    +  def withNewPoll(
    +      records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +      offsetAfterPoll: Long): FetchedData = {
    +    this._records = records
    +    this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    +    this._offsetAfterPoll = offsetAfterPoll
    +    this
    +  }
     
    -    /** Move `records` backward and return the previous record. */
    -    def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      assert(_records.hasPrevious, "fetchedData cannot move back")
    -      val record = _records.previous()
    -      _nextOffsetInFetchedData = record.offset
    -      record
    -    }
    +  /** Whether there are more elements */
    +  def hasNext: Boolean = _records.hasNext
     
    -    /** Reset the internal pre-fetched data. */
    -    def reset(): Unit = {
    -      _records = ju.Collections.emptyListIterator()
    -    }
    +  /** Move `records` forward and return the next record. */
    +  def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    val record = _records.next()
    +    _nextOffsetInFetchedData = record.offset + 1
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset in `records`. We use this to verify if we should check if the
    -     * pre-fetched data is still valid.
    -     */
    -    def nextOffsetInFetchedData: Long = _nextOffsetInFetchedData
    +  /** Move `records` backward and return the previous record. */
    +  def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    assert(_records.hasPrevious, "fetchedData cannot move back")
    +    val record = _records.previous()
    +    _nextOffsetInFetchedData = record.offset
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset to poll after draining the pre-fetched records.
    -     */
    -    def offsetAfterPoll: Long = _offsetAfterPoll
    +  /** Reset the internal pre-fetched data. */
    +  def reset(): Unit = {
    +    _records = ju.Collections.emptyListIterator()
       }
     
       /**
    -   * The internal object returned by the `fetchRecord` method. If `record` is empty, it means it is
    -   * invisible (either a transaction message, or an aborted message when the consumer's
    -   * `isolation.level` is `read_committed`), and the caller should use `nextOffsetToFetch` to fetch
    -   * instead.
    +   * Returns the next offset in `records`. We use this to verify if we should check if the
    +   * pre-fetched data is still valid.
        */
    -  private case class FetchedRecord(
    -      var record: ConsumerRecord[Array[Byte], Array[Byte]],
    -      var nextOffsetToFetch: Long) {
    -
    -    def withRecord(
    -        record: ConsumerRecord[Array[Byte], Array[Byte]],
    -        nextOffsetToFetch: Long): FetchedRecord = {
    -      this.record = record
    -      this.nextOffsetToFetch = nextOffsetToFetch
    -      this
    -    }
    -  }
    +  def nextOffsetInFetchedData: Long = _nextOffsetInFetchedData
     
    -  private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +  /**
    +   * Returns the next offset to poll after draining the pre-fetched records.
    +   */
    +  def offsetAfterPoll: Long = _offsetAfterPoll
    +}
     
    -  @volatile private var consumer = createConsumer
    +/**
    + * The internal object returned by the `fetchRecord` method. If `record` is empty, it means it is
    + * invisible (either a transaction message, or an aborted message when the consumer's
    + * `isolation.level` is `read_committed`), and the caller should use `nextOffsetToFetch` to fetch
    + * instead.
    + */
    +private[kafka010] case class FetchedRecord(
    +    var record: ConsumerRecord[Array[Byte], Array[Byte]],
    +    var nextOffsetToFetch: Long) {
    +
    +  def withRecord(
    +      record: ConsumerRecord[Array[Byte], Array[Byte]],
    +      nextOffsetToFetch: Long): FetchedRecord = {
    +    this.record = record
    +    this.nextOffsetToFetch = nextOffsetToFetch
    +    this
    +  }
    +}
     
    -  /** indicates whether this consumer is in use or not */
    -  @volatile var inUse = true
    +/**
    + * This class helps caller to read from Kafka leveraging consumer pool as well as fetched data pool.
    + * This class throws error when data loss is detected while reading from Kafka.
    + *
    + * NOTE for contributors: we need to ensure all the public methods to initialize necessary resources
    + * via calling `ensureConsumerAvailable` and `ensureFetchedDataAvailable`.
    + */
    +private[kafka010] class KafkaDataConsumer(
    +    topicPartition: TopicPartition,
    +    kafkaParams: ju.Map[String, Object],
    +    consumerPool: InternalKafkaConsumerPool,
    +    fetchedDataCache: FetchedDataPool) extends Logging {
    --- End diff --
    
    Sometimes called `fetchedDataPool` and sometimes `fetchedDataCache`. It would be good to call one way.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #98909 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98909/testReport)** for PR 22138 at commit [`fd4ff83`](https://github.com/apache/spark/commit/fd4ff833b6c2b5889d55ee4053970b56ee2b273d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95784 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95784/testReport)** for PR 22138 at commit [`2da9924`](https://github.com/apache/spark/commit/2da99247da82036abec7dfbb584f9bb7ff99c577).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Seeking means the pre-fetched data is wasted, so it's not a light
    operation.  It shouldn't be unavoidable, e.g. if consumers were cached
    keyed by topicpartition, groupid, next offset to be processed.  One concern
    there would be how to make sure you don't have lots of idle consumers.
    
    The question of how serious an issue is could be solved by measurement, but
    I don't have production structured streaming jobs, much less ones that
    exhibit the kind of behavior tdas was talking about in the original ticket.
    
    On Mon, Aug 20, 2018 at 7:36 PM, Jungtaek Lim <no...@github.com>
    wrote:
    
    > @koeninger <https://github.com/koeninger>
    >
    > I'm not sure but are you saying that an executor cares about multiple
    > queries (multiple jobs) concurrently? I honestly didn't notice it. If that
    > is going to be problem, we should add something (could we get query id at
    > that time?) in cache key to differentiate consumers. If we want to avoid
    > extra seeking due to different offsets, consumers should not be reused
    > among with multiple queries, and that's just a matter of cache key.
    >
    > If you are thinking about co-use of consumers among multiple queries
    > because of reusing connection to Kafka, I think extra seeking is
    > unavoidable (I guess fetched data should be much more critical issue unless
    > we never reuse after returning to pool). If seeking is light operation, we
    > may even go with only reusing connection (not position we already sought):
    > always resetting position (and data maybe?) when borrowing from pool or
    > returning consumer to pool.
    >
    > Btw, the rationalization of this patch is not solving the issue you're
    > referring. This patch is also based on #20767
    > <https://github.com/apache/spark/pull/20767> but dealing with another
    > improvements pointed out in comments: adopt pool library to not reinvent
    > the wheel, and also enabling metrics regarding the pool.
    >
    > I'm not sure the issue you're referring is a serious one (show-stopper):
    > if the issue is a kind of serious, someone should handle the issue once we
    > are aware of the issue at March, or at least relevant JIRA issue should be
    > filed with detailed explanation before. I'd like to ask you in favor of
    > handling (or filing) the issue since you may know the issue best.
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/22138#issuecomment-414498067>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AAGAB7qFjFrj9dWWkIcUcKcAKbEicuOwks5uS0gDgaJpZM4WCUJs>
    > .
    >



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94914/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this, please


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #98904 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98904/testReport)** for PR 22138 at commit [`fd4ff83`](https://github.com/apache/spark/commit/fd4ff833b6c2b5889d55ee4053970b56ee2b273d).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Updated the description of PR to reflect the new approach.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95525 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95525/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95502/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95513 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95513/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    If my understanding is right, looks like current approach has same limitation. I guess you're busy, but could you refer some issue number or point out some code lines which was based on the reason if you remember any? It should help to determine whether this patch breaks more spots or not.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r211119914
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -425,70 +381,36 @@ private[kafka010] object KafkaDataConsumer extends Logging {
       def acquire(
           topicPartition: TopicPartition,
           kafkaParams: ju.Map[String, Object],
    -      useCache: Boolean): KafkaDataConsumer = synchronized {
    -    val key = new CacheKey(topicPartition, kafkaParams)
    -    val existingInternalConsumer = cache.get(key)
    +      useCache: Boolean): KafkaDataConsumer = {
     
    -    lazy val newInternalConsumer = new InternalKafkaConsumer(topicPartition, kafkaParams)
    +    if (!useCache) {
    +      return NonCachedKafkaDataConsumer(new InternalKafkaConsumer(topicPartition, kafkaParams))
    +    }
     
    -    if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) {
    -      // If this is reattempt at running the task, then invalidate cached consumer if any and
    -      // start with a new one.
    -      if (existingInternalConsumer != null) {
    -        // Consumer exists in cache. If its in use, mark it for closing later, or close it now.
    -        if (existingInternalConsumer.inUse) {
    -          existingInternalConsumer.markedForClose = true
    -        } else {
    -          existingInternalConsumer.close()
    -        }
    -      }
    -      cache.remove(key)  // Invalidate the cache in any case
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    val key = new CacheKey(topicPartition, kafkaParams)
     
    -    } else if (!useCache) {
    -      // If planner asks to not reuse consumers, then do not use it, return a new consumer
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumer if any.
     
    -    } else if (existingInternalConsumer == null) {
    -      // If consumer is not already cached, then put a new in the cache and return it
    -      cache.put(key, newInternalConsumer)
    -      newInternalConsumer.inUse = true
    -      CachedKafkaDataConsumer(newInternalConsumer)
    +      // invalidate all idle consumers for the key
    +      pool.invalidateKey(key)
     
    -    } else if (existingInternalConsumer.inUse) {
    -      // If consumer is already cached but is currently in use, then return a new consumer
    -      NonCachedKafkaDataConsumer(newInternalConsumer)
    +      // borrow a consumer from pool even in this case
    --- End diff --
    
    This is another behavior change: If this attempt succeeds, we can use pooled consumer from next batch, so no reason to discard the consumer.
    
    But I also see the cost of unnecessary pooling if failure occurs continuously.
    
    So that looks like kind of decision between possibility of success vs possibility of failure (again), and while I decide to cache it, it is pretty easy to go back to current behavior, so please let me know if we think current behavior makes more sense.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Rebased to resolve conflict.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96317 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96317/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Just rebased.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    see e.g. https://github.com/apache/spark/pull/20767 for background
    
    Even if this patch doesn't change behavior, if it doesn't really solve the
    problem, it may make it harder to solve it correctly.
    
    On Mon, Aug 20, 2018 at 10:32 AM, Jungtaek Lim <no...@github.com>
    wrote:
    
    > If my understanding is right, looks like current approach has same
    > limitation. I guess you're busy, but could you refer some issue number or
    > point out some code lines which was based on the reason if you remember
    > any? It should help to determine whether this patch breaks more spots or
    > not.
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/22138#issuecomment-414336510>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AAGAB52zi78tff9E-kvPzeB7k0ZnOyFBks5uSsh0gaJpZM4WCUJs>
    > .
    >



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94913 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94913/testReport)** for PR 22138 at commit [`94231fe`](https://github.com/apache/spark/commit/94231fef1f2f59cea1625fd1f71bd99372a8e800).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96300 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96300/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214800822
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    +      setMaxTotal(-1)
    +
    +      // Set minimum evictable idle time which will be referred from evictor thread
    +      setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis)
    +      setSoftMinEvictableIdleTimeMillis(-1)
    +
    +      // evictor thread will run test with ten idle objects
    +      setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis)
    +      setNumTestsPerEvictionRun(10)
    +      setEvictionPolicy(new DefaultEvictionPolicy[InternalKafkaConsumer]())
    +
    +      // Immediately fail on exhausted pool while borrowing
    +      setBlockWhenExhausted(false)
    +
    +      setJmxEnabled(jmxEnabled)
    +      setJmxNamePrefix("kafka010-cached-simple-kafka-consumer-pool")
    +    }
    +  }
    +
    +  object PoolConfig {
    +    val CONFIG_NAME_PREFIX = "spark.sql.kafkaConsumerCache."
    +    val CONFIG_NAME_CAPACITY = CONFIG_NAME_PREFIX + "capacity"
    +    val CONFIG_NAME_JMX_ENABLED = CONFIG_NAME_PREFIX + "jmx.enable"
    +    val CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS = CONFIG_NAME_PREFIX +
    +      "minEvictableIdleTimeMillis"
    +    val CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = CONFIG_NAME_PREFIX +
    +      "evictorThreadRunIntervalMillis"
    +
    +    val DEFAULT_VALUE_CAPACITY = 64
    +    val DEFAULT_VALUE_JMX_ENABLED = false
    +    val DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS = 5 * 60 * 1000 // 5 minutes
    +    val DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = 3 * 60 * 1000 // 3 minutes
    +  }
    +
    +  class ObjectFactory extends BaseKeyedPooledObjectFactory[CacheKey, InternalKafkaConsumer]
    +    with Logging {
    +
    +    val keyToKafkaParams: ConcurrentHashMap[CacheKey, ju.Map[String, Object]] =
    +      new ConcurrentHashMap[CacheKey, ju.Map[String, Object]]()
    +
    +    override def create(key: CacheKey): InternalKafkaConsumer = {
    +      val kafkaParams = keyToKafkaParams.get(key)
    +      if (kafkaParams == null) {
    --- End diff --
    
    Why not using option?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95777/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95450/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215313215
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    To make clear once again, was your finding a false alarm, or there's something to fix which I can reproduce the issue easily?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    I just addressed eviction to consumer pool as well as added relevant test. This will help closing invalid idle consumers which topic or partition are no longer be assigned to any tasks. I guess current cache is not capable of closing invalid consumers.
    
    I haven't find how to add "query id" to the cache key, but IMHO the patch itself is already providing some values to be merged. It would be even better if someone could guide how to add "query id" to the cache key.
    
    @koeninger @tdas @zsxwing Please take a look and comment. Thanks in advance!


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214917336
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -414,17 +468,37 @@ private[kafka010] case class InternalKafkaConsumer(
         }
       }
     
    -  /** Create a new consumer and reset cached states */
    -  private def resetConsumer(): Unit = {
    -    consumer.close()
    -    consumer = createConsumer
    -    fetchedData.reset()
    +  /**
    +   * Poll messages from Kafka starting from `offset` and update `fetchedData`. `fetchedData` may be
    +   * empty if the Kafka consumer fetches some messages but all of them are not visible messages
    +   * (either transaction messages, or aborted messages when `isolation.level` is `read_committed`).
    +   *
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
    +   */
    +  private def fetchData(offset: Long, pollTimeoutMs: Long): Unit = {
    +    val (records, offsetAfterPoll) = consumer.fetch(offset, pollTimeoutMs)
    +    fetchedData.withNewPoll(records.listIterator, offsetAfterPoll)
    +  }
    +
    +  private def ensureConsumerAvailable(): Unit = {
    +    if (consumer == null) {
    +      consumer = consumerPool.borrowObject(cacheKey, kafkaParams)
    +    }
    +  }
    +
    +  private def ensureFetchedDataAvailable(offset: Long): Unit = {
    +    if (fetchedData == null) {
    --- End diff --
    
    Same here.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by attilapiros <gi...@git.apache.org>.
Github user attilapiros commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213621710
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    +    // set invalidate timestamp to let active objects being destroyed when returning to pool
    +    objectFactory.keyToLastInvalidatedTimestamp.put(key, System.currentTimeMillis())
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(intConsumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(intConsumer.topicPartition, intConsumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = e match {
    +      case e1: PooledObjectInvalidated =>
    +        logDebug("Pool for key was invalidated after cached object was borrowed. " +
    +          s"Invalidating cached object - key: ${e1.key} / borrowed timestamp: " +
    +          s"${e1.lastBorrowedTime} / invalidated timestamp for key: ${e1.lastInvalidatedTimestamp}")
    +
    +      case _ => logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +      val capacity = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior of CachedInternalKafkaConsumerPool, so do not modify
    --- End diff --
    
    I have not found the referenced CachedInternalKafkaConsumerPool. I guess you mean InternalKafkaConsumerPool.  Isn't it?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214717195
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    --- End diff --
    
    Nit: `executorService.scheduleAtFixedRate(new Runnable {` is enough.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95474 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95474/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95450 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95450/testReport)** for PR 22138 at commit [`7d13ee5`](https://github.com/apache/spark/commit/7d13ee5601c6e71200700c44de067325aa057f9a).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215277752
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    I've just put `throw SparkException("foo")` into `removeIdleFetchedData` and started the suite.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    One more thing just came to my mind is the documentation. The parameter documentation is a gap even for the original feature. As it has been grown and several additional parameters added it would be good to document them on the kafka integration page.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96695 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96695/testReport)** for PR 22138 at commit [`d3f097b`](https://github.com/apache/spark/commit/d3f097bcd2c808a5433444426a6990b217774f86).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214721690
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    +      override def run(): Unit = {
    +        removeIdleFetchedData()
    --- End diff --
    
    Any thrown exception or error reaching the executor causes the executor to halt.
    `catch` + `log...` would be good.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218699852
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    Works fine, thanks!


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    cc. @tdas @zsxwing @koeninger @arunmahadevan 


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by attilapiros <gi...@git.apache.org>.
Github user attilapiros commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213639419
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    +    // set invalidate timestamp to let active objects being destroyed when returning to pool
    +    objectFactory.keyToLastInvalidatedTimestamp.put(key, System.currentTimeMillis())
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(intConsumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(intConsumer.topicPartition, intConsumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = e match {
    +      case e1: PooledObjectInvalidated =>
    +        logDebug("Pool for key was invalidated after cached object was borrowed. " +
    +          s"Invalidating cached object - key: ${e1.key} / borrowed timestamp: " +
    +          s"${e1.lastBorrowedTime} / invalidated timestamp for key: ${e1.lastInvalidatedTimestamp}")
    +
    +      case _ => logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +      val capacity = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    --- End diff --
    
    Is there a common place where we should document these configurations?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96695/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215268638
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    --- End diff --
    
    Then `PrivateMethodTester` can be used.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96300/
    Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215818860
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    It looks like reaching `eventually`. What I've missed actually is, specify wrong time unit in overall timeout in `eventually` so it waits for 1000x of expected time. After fixing that I can see test failure around 2 seconds.
    
    ```
    The code passed to eventually never returned normally. Attempted 13 times over 2.016476764 seconds. Last failure message: ListBuffer(CachedFetchedData(FetchedData(java.util.AbstractList$ListItr@7426a448,-2,5))) was not empty.
    ```
    
    Thanks for finding the bug! Will fix.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96317 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96317/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213866399
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    --- End diff --
    
    It is to reduce unnecessary computation along with comparing map while accessing with pool. You can see CacheKey keeps as it is, and I guess CacheKey was designed for same reason.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96695 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96695/testReport)** for PR 22138 at commit [`d3f097b`](https://github.com/apache/spark/commit/d3f097bcd2c808a5433444426a6990b217774f86).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215579562
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    Now I see how the seeking issue solved, good. I think it's highly welcome and essential in terms of performance which would be good to cover at least in one test (made a toy test and in logs I see it works). Something like:
    ```
    createTopicWithOnePartition
    get(0)
    assert seek
    get(500)
    assert seek
    get(1)
    assert no seek
    ```



---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218955883
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    Let's see how committers think about it. We can wait and react.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Retest this, please.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215635068
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    Actually soft boundary with existing cache is more like total keys in new cache (and new cache implements it), while you're concerning about it is regarding same key having too many tasks hence too many consumers.
    
    In new cache we have more metrics: the number of idle/active consumers for given key, the number of total idle/active consumers. We can check the metrics and log when it exceeds the threshold or meet condition. Even simpler, we can just leave log message whenever cache **creates** consumer instead of returning idle consumer, if it would help end users to tune pool as well as tasks.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214705613
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    This has to be bounded somehow with a configuration. I can't imagine it will be efficient if for instance more than 16 tasks read the same topicPartitions.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    No objection about `org.apache.commons.pool2.impl.GenericKeyedObjectPool` stuff but just wondering why `org.apache.spark.sql.kafka010.CachedKafkaProducer` uses `com.google.common.cache.LoadingCache`?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96295 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96295/testReport)** for PR 22138 at commit [`da5bd7f`](https://github.com/apache/spark/commit/da5bd7fd79a0ce41ca7301ecdc4e375c897e8036).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95525/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95206 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95206/testReport)** for PR 22138 at commit [`859a9d0`](https://github.com/apache/spark/commit/859a9d0b3cb7dba7fbd727af78d9a88072501c0d).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98904/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this, please


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this, please


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95394 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95394/testReport)** for PR 22138 at commit [`ba576e8`](https://github.com/apache/spark/commit/ba576e88578b50897dd73b385f3f2308976c088a).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94913/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    From the stack trace of test failure, it doesn't look like relevant to the code change I guess. Jenkins shows REGRESSION but the test is added at Aug 25, 2018 so aged less than 7 days, which is not enough to consider it as regression or not I think. The test itself could be flaky. The test succeeds 5 times in a row in local dev. and I'll try to run more. 
    
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95474/testReport/org.apache.spark.sql.kafka010/KafkaSourceStressForDontFailOnDataLossSuite/stress_test_for_failOnDataLoss_false/
    
    @zsxwing Do you have any idea on this? I'm curious how checking query exception after query.stop() is relevant to `failOnDataLoss = false`, given that we check query exception all the time in loop.
    
    https://github.com/apache/spark/blob/bb3e6ed9216f98f3a3b96c8c52f20042d65e2181/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala#L270-L274
    
    Will retrigger build for now.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95784/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95777 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95777/testReport)** for PR 22138 at commit [`5b56bd0`](https://github.com/apache/spark/commit/5b56bd0e26e198281ebe6888a8f012efe325cdab).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214910337
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    +      override def run(): Unit = {
    +        removeIdleFetchedData()
    --- End diff --
    
    Nice catch! Will address.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214910433
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    --- End diff --
    
    Will address.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214806388
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    --- End diff --
    
    Shouldn't it be `FetchedDataPoolSuite.scala`?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215594790
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    As I see it never reaches `eventually`.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @koeninger 
    I'm not sure I got your point correctly. This patch is based on some assumptions, so please correct me if I'm missing here. Assumptions follow:
    
    1. There's actually no multiple consumers for a given key working at the same time. The cache key contains topic partition as well as group id. Even the query tries to do self-join so reading same topic in two different sources, I think group id should be different.
    
    2. In normal case the offset will be continuous, and that's why cache should help. In retrying case this patch invalidates cache as same as current behavior, so it should start from scratch.
    
    (Btw, I'm curious what's more expensive between `leveraging pooled object but resetting kafka consumer` vs `invalidating pooled objects and start from scratch`. Latter feels more safer but if we just need extra seek instead of reconnecting to kafka, resetting could be improved and former will be cheaper. I feel it is out of scope of my PR though.)
    
    This patch keeps most of current behaviors, except two spots I guess. I already commented a spot why I change the behavior, and I'll comment another spot for the same.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96317/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95744 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95744/testReport)** for PR 22138 at commit [`b47e99b`](https://github.com/apache/spark/commit/b47e99b82e13968b058c9f90ab3e413f5c563ee6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215583862
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    That said just put `throw new SparkException("foo")` at the beginning of `removeIdleFetchedData ` function and the mentioned test never fails. With the new try-catch block one can see the exception but not timing out or proceeding at all.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95394/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this, please


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215274047
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    +      setMaxTotal(-1)
    +
    +      // Set minimum evictable idle time which will be referred from evictor thread
    +      setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis)
    +      setSoftMinEvictableIdleTimeMillis(-1)
    +
    +      // evictor thread will run test with ten idle objects
    +      setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis)
    +      setNumTestsPerEvictionRun(10)
    +      setEvictionPolicy(new DefaultEvictionPolicy[InternalKafkaConsumer]())
    +
    +      // Immediately fail on exhausted pool while borrowing
    +      setBlockWhenExhausted(false)
    +
    +      setJmxEnabled(jmxEnabled)
    +      setJmxNamePrefix("kafka010-cached-simple-kafka-consumer-pool")
    +    }
    +  }
    +
    +  object PoolConfig {
    +    val CONFIG_NAME_PREFIX = "spark.sql.kafkaConsumerCache."
    +    val CONFIG_NAME_CAPACITY = CONFIG_NAME_PREFIX + "capacity"
    +    val CONFIG_NAME_JMX_ENABLED = CONFIG_NAME_PREFIX + "jmx.enable"
    +    val CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS = CONFIG_NAME_PREFIX +
    +      "minEvictableIdleTimeMillis"
    +    val CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = CONFIG_NAME_PREFIX +
    +      "evictorThreadRunIntervalMillis"
    +
    +    val DEFAULT_VALUE_CAPACITY = 64
    +    val DEFAULT_VALUE_JMX_ENABLED = false
    +    val DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS = 5 * 60 * 1000 // 5 minutes
    +    val DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = 3 * 60 * 1000 // 3 minutes
    +  }
    +
    +  class ObjectFactory extends BaseKeyedPooledObjectFactory[CacheKey, InternalKafkaConsumer]
    +    with Logging {
    +
    +    val keyToKafkaParams: ConcurrentHashMap[CacheKey, ju.Map[String, Object]] =
    +      new ConcurrentHashMap[CacheKey, ju.Map[String, Object]]()
    +
    +    override def create(key: CacheKey): InternalKafkaConsumer = {
    +      val kafkaParams = keyToKafkaParams.get(key)
    +      if (kafkaParams == null) {
    --- End diff --
    
    It's more a beautify thing then an issue. Using `Option` is more scala idiomatic then checking null.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214918569
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    I'm not sure I'm seeing the scenario you're referring. Could you elaborate? Don't `eventually` and `assert` avoid such situation?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213867027
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    +    // set invalidate timestamp to let active objects being destroyed when returning to pool
    +    objectFactory.keyToLastInvalidatedTimestamp.put(key, System.currentTimeMillis())
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(intConsumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(intConsumer.topicPartition, intConsumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = e match {
    +      case e1: PooledObjectInvalidated =>
    +        logDebug("Pool for key was invalidated after cached object was borrowed. " +
    +          s"Invalidating cached object - key: ${e1.key} / borrowed timestamp: " +
    +          s"${e1.lastBorrowedTime} / invalidated timestamp for key: ${e1.lastInvalidatedTimestamp}")
    +
    +      case _ => logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +      val capacity = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    --- End diff --
    
    Hmm... I think `spark.sql.kafkaConsumerCache.capacity` wasn't documented somewhere, but it may be the thing to fix. The patch will provide more configurations to let end users can tune, so maybe worth to add them to some docs? Not 100% sure where to.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214817471
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    +      override def run(): Unit = {
    +        removeIdleFetchedData()
    +      }
    +    }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS)
    +  }
    +
    +  startEvictorThread()
    +
    +  def acquire(key: CacheKey, desiredStartOffset: Long): FetchedData = synchronized {
    +    val fetchedDataList = cache.getOrElseUpdate(key, new CachedFetchedDataList())
    +
    +    val cachedFetchedDataOption = fetchedDataList.find { p =>
    +      !p.inUse && p.getObject.nextOffsetInFetchedData == desiredStartOffset
    +    }
    +
    +    var cachedFetchedData: CachedFetchedData = null
    +    if (cachedFetchedDataOption.isDefined) {
    +      cachedFetchedData = cachedFetchedDataOption.get
    +    } else {
    +      cachedFetchedData = CachedFetchedData.empty()
    +      fetchedDataList += cachedFetchedData
    +    }
    +
    +    cachedFetchedData.lastAcquiredTimestamp = System.currentTimeMillis()
    +    cachedFetchedData.inUse = true
    +
    +    cachedFetchedData.getObject
    +  }
    +
    +  def invalidate(key: CacheKey): Unit = synchronized {
    +    cache.remove(key)
    +  }
    +
    +  def release(key: CacheKey, fetchedData: FetchedData): Unit = synchronized {
    +    cache.get(key) match {
    +      case Some(fetchedDataList) =>
    +        val cachedFetchedDataOption = fetchedDataList.find { p =>
    +          p.inUse && p.getObject == fetchedData
    +        }
    +
    +        if (cachedFetchedDataOption.isDefined) {
    +          val cachedFetchedData = cachedFetchedDataOption.get
    +          cachedFetchedData.inUse = false
    +          cachedFetchedData.lastReleasedTimestamp = System.currentTimeMillis()
    +        }
    +
    +      case None =>
    --- End diff --
    
    Some warning would be good because it could be a programming failure.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95474/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95744 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95744/testReport)** for PR 22138 at commit [`b47e99b`](https://github.com/apache/spark/commit/b47e99b82e13968b058c9f90ab3e413f5c563ee6).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94912 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94912/testReport)** for PR 22138 at commit [`c82f306`](https://github.com/apache/spark/commit/c82f3064fa8744f91b5c8a92645588dc9d53ba35).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95474 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95474/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95525 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95525/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218725548
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    In my view the mentioned simple debug log message is fine. This can be seen even without constant monitoring.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96295/
    Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218777053
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,243 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    val oldKafkaParams = objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +    require(oldKafkaParams == null || kafkaParams == oldKafkaParams, "Kafka parameters for same " +
    +      s"cache key should be equal. old parameters: $oldKafkaParams new parameters: $kafkaParams")
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    --- End diff --
    
    That was used from previous approach. Thanks for finding. Will remove.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215591546
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    Yeah, the first approach is heavy and doesn't solve the problem itself. The second case I'm not sure how this can help (maybe not understood properly). Additionally it could introduce fluctuation in the performance. Maybe we can leave the soft boundary (as it's like that in the original implementation and I don't see jiras with it) and just put warning on each new allocation after a threshold. This way admins/devs can see this in the log and don't have to monitor the cache constantly.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95744/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214803892
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -414,17 +468,37 @@ private[kafka010] case class InternalKafkaConsumer(
         }
       }
     
    -  /** Create a new consumer and reset cached states */
    -  private def resetConsumer(): Unit = {
    -    consumer.close()
    -    consumer = createConsumer
    -    fetchedData.reset()
    +  /**
    +   * Poll messages from Kafka starting from `offset` and update `fetchedData`. `fetchedData` may be
    +   * empty if the Kafka consumer fetches some messages but all of them are not visible messages
    +   * (either transaction messages, or aborted messages when `isolation.level` is `read_committed`).
    +   *
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
    +   */
    +  private def fetchData(offset: Long, pollTimeoutMs: Long): Unit = {
    +    val (records, offsetAfterPoll) = consumer.fetch(offset, pollTimeoutMs)
    +    fetchedData.withNewPoll(records.listIterator, offsetAfterPoll)
    +  }
    +
    +  private def ensureConsumerAvailable(): Unit = {
    +    if (consumer == null) {
    +      consumer = consumerPool.borrowObject(cacheKey, kafkaParams)
    +    }
    +  }
    +
    +  private def ensureFetchedDataAvailable(offset: Long): Unit = {
    +    if (fetchedData == null) {
    --- End diff --
    
    Why not using option?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94913 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94913/testReport)** for PR 22138 at commit [`94231fe`](https://github.com/apache/spark/commit/94231fef1f2f59cea1625fd1f71bd99372a8e800).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214917284
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -414,17 +468,37 @@ private[kafka010] case class InternalKafkaConsumer(
         }
       }
     
    -  /** Create a new consumer and reset cached states */
    -  private def resetConsumer(): Unit = {
    -    consumer.close()
    -    consumer = createConsumer
    -    fetchedData.reset()
    +  /**
    +   * Poll messages from Kafka starting from `offset` and update `fetchedData`. `fetchedData` may be
    +   * empty if the Kafka consumer fetches some messages but all of them are not visible messages
    +   * (either transaction messages, or aborted messages when `isolation.level` is `read_committed`).
    +   *
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
    +   */
    +  private def fetchData(offset: Long, pollTimeoutMs: Long): Unit = {
    +    val (records, offsetAfterPoll) = consumer.fetch(offset, pollTimeoutMs)
    +    fetchedData.withNewPoll(records.listIterator, offsetAfterPoll)
    +  }
    +
    +  private def ensureConsumerAvailable(): Unit = {
    +    if (consumer == null) {
    --- End diff --
    
    This is defined as `var` so just to avoid additional wrapping here. Same here as above: if we prefer Option I'm happy to change but not sure about it.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Thanks @attilapiros I fixed the missing spots you left comments.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96337 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96337/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    I thought the whole reason the caching was changed from the initial naive
    approach to the current approach in master was that people were running
    jobs that were scheduling multiple consumers for the same topicpartition
    and group.
    
    
    
    On Sun, Aug 19, 2018 at 7:51 PM, Jungtaek Lim <no...@github.com>
    wrote:
    
    > @koeninger <https://github.com/koeninger>
    > I'm not sure I got your point correctly. This patch is based on some
    > assumptions, so please correct me if I'm missing here. Assumptions follow:
    >
    >    1.
    >
    >    There's actually no multiple consumers for a given key working at the
    >    same time. The cache key contains topic partition as well as group id. Even
    >    the query tries to do self-join so reading same topic in two different
    >    sources, I think group id should be different.
    >    2.
    >
    >    In normal case the offset will be continuous, and that's why cache
    >    should help. In retrying case this patch invalidates cache as same as
    >    current behavior, so it should start from scratch.
    >
    > (Btw, I'm curious what's more expensive between leveraging pooled object
    > but resetting kafka consumer vs invalidating pooled objects and start
    > from scratch. Latter feels more safer but if we just need extra seek
    > instead of reconnecting to kafka, resetting could be improved and former
    > will be cheaper. I feel it is out of scope of my PR though.)
    >
    > This patch keeps most of current behaviors, except two spots I guess. I
    > already commented a spot why I change the behavior, and I'll comment
    > another spot for the same.
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/22138#issuecomment-414164788>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AAGAB8x3Khz4bWIxphLJHWFvcc8H4ERyks5uSfnvgaJpZM4WCUJs>
    > .
    >



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by attilapiros <gi...@git.apache.org>.
Github user attilapiros commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213869998
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    --- End diff --
    
    Oh I see. What about changing CacheKey to not a case class but to a class where kafkaParams is a member but its equals and hashCode methods does not use kafkaParams?
    As this values are goes together I have the feeling encapsulating them is better then keeping their relation in a separate map (keyToKafkaParams). It is just an idea.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95206/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95472 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95472/testReport)** for PR 22138 at commit [`648550a`](https://github.com/apache/spark/commit/648550ad7028b5576457caf8bbcac0e415366674).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215637613
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    I guess we already have `multiple tasks referring same key continuously using fetched data` if we just want to verify correctness, though checking with KafkaDataConsumer may worth it in terms of acceptance test. `seek` itself doesn't impact huge on performance. `Discarding fetched data due to offset mismatch` would hugely impact on performance, so this should be checked if we would want to be sure about improvement.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r219367280
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    +    private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +    private var _nextOffsetInFetchedData: Long,
    +    private var _offsetAfterPoll: Long) {
    +
    +  def withNewPoll(
    +      records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +      offsetAfterPoll: Long): FetchedData = {
    +    this._records = records
    +    this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    +    this._offsetAfterPoll = offsetAfterPoll
    +    this
    +  }
     
    -    /** Move `records` backward and return the previous record. */
    -    def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      assert(_records.hasPrevious, "fetchedData cannot move back")
    -      val record = _records.previous()
    -      _nextOffsetInFetchedData = record.offset
    -      record
    -    }
    +  /** Whether there are more elements */
    +  def hasNext: Boolean = _records.hasNext
     
    -    /** Reset the internal pre-fetched data. */
    -    def reset(): Unit = {
    -      _records = ju.Collections.emptyListIterator()
    -    }
    +  /** Move `records` forward and return the next record. */
    +  def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    val record = _records.next()
    +    _nextOffsetInFetchedData = record.offset + 1
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset in `records`. We use this to verify if we should check if the
    -     * pre-fetched data is still valid.
    -     */
    -    def nextOffsetInFetchedData: Long = _nextOffsetInFetchedData
    +  /** Move `records` backward and return the previous record. */
    +  def previous(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    assert(_records.hasPrevious, "fetchedData cannot move back")
    +    val record = _records.previous()
    +    _nextOffsetInFetchedData = record.offset
    +    record
    +  }
     
    -    /**
    -     * Returns the next offset to poll after draining the pre-fetched records.
    -     */
    -    def offsetAfterPoll: Long = _offsetAfterPoll
    +  /** Reset the internal pre-fetched data. */
    +  def reset(): Unit = {
    +    _records = ju.Collections.emptyListIterator()
    --- End diff --
    
    Should reflect #22507 when #22507 is merged.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    retest this, please


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Thanks for your PR. This is really a big change. It will need very careful review as it changes a lot of critical code path and the current Kafka consumer logic is really complicated. Let's hold this before 2.4 branch gets cut as it's risky to put this into 2.4. It's basically blocked by a complicated correctness fix and should be fixed soon.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95688/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214813543
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    --- End diff --
    
    Then why not enforce it?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96250 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96250/testReport)** for PR 22138 at commit [`d2f07b7`](https://github.com/apache/spark/commit/d2f07b7796f281c6e020a0f2dcabb9f4a7c37a75).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @zsxwing 
    Given that Spark 2.4 vote passes, could we revisit and make progress on this?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95394 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95394/testReport)** for PR 22138 at commit [`ba576e8`](https://github.com/apache/spark/commit/ba576e88578b50897dd73b385f3f2308976c088a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95502 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95502/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96337 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96337/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by attilapiros <gi...@git.apache.org>.
Github user attilapiros commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213615086
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    --- End diff --
    
    Why kafkaParams is passed as the second argument? 
    As I see CacheKey itself is constructed from kafkaParams so would not be better to store kafkaParam in a val within CacheKey? 
    
    Then objectFactory.keyToKafkaParams would be deleted along with updateKafkaParamForKey. Is not it?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96250/
    Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95688 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95688/testReport)** for PR 22138 at commit [`9685cc5`](https://github.com/apache/spark/commit/9685cc59255458c41d293da4117954cc653fab28).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215867141
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    Just added some tests on KafkaDataConsumer, basic sanity as well as edge cases.
    Please take a look at 4e2790a2cb6eff1b78af0f5a01946dad29b0b2ad



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96300 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96300/testReport)** for PR 22138 at commit [`ddd4f2f`](https://github.com/apache/spark/commit/ddd4f2fc38c42dd1b781b0e3df46432bb6829e7b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94914 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94914/testReport)** for PR 22138 at commit [`fd728ef`](https://github.com/apache/spark/commit/fd728ef8c99ebb33d6dba5466e6a8dba8984248d).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95800 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95800/testReport)** for PR 22138 at commit [`107880e`](https://github.com/apache/spark/commit/107880e0e7fb946a0ebf838739320699be968083).


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #98904 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98904/testReport)** for PR 22138 at commit [`fd4ff83`](https://github.com/apache/spark/commit/fd4ff833b6c2b5889d55ee4053970b56ee2b273d).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215310403
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    --- End diff --
    
    Ah yes, wasn't aware of PrivateMethodTester. Thanks! Will apply.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214909826
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    +      override def run(): Unit = {
    +        removeIdleFetchedData()
    +      }
    +    }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS)
    +  }
    +
    +  startEvictorThread()
    +
    +  def acquire(key: CacheKey, desiredStartOffset: Long): FetchedData = synchronized {
    +    val fetchedDataList = cache.getOrElseUpdate(key, new CachedFetchedDataList())
    +
    +    val cachedFetchedDataOption = fetchedDataList.find { p =>
    +      !p.inUse && p.getObject.nextOffsetInFetchedData == desiredStartOffset
    +    }
    +
    +    var cachedFetchedData: CachedFetchedData = null
    +    if (cachedFetchedDataOption.isDefined) {
    +      cachedFetchedData = cachedFetchedDataOption.get
    +    } else {
    +      cachedFetchedData = CachedFetchedData.empty()
    +      fetchedDataList += cachedFetchedData
    +    }
    +
    +    cachedFetchedData.lastAcquiredTimestamp = System.currentTimeMillis()
    +    cachedFetchedData.inUse = true
    +
    +    cachedFetchedData.getObject
    +  }
    +
    +  def invalidate(key: CacheKey): Unit = synchronized {
    +    cache.remove(key)
    +  }
    +
    +  def release(key: CacheKey, fetchedData: FetchedData): Unit = synchronized {
    +    cache.get(key) match {
    +      case Some(fetchedDataList) =>
    +        val cachedFetchedDataOption = fetchedDataList.find { p =>
    +          p.inUse && p.getObject == fetchedData
    +        }
    +
    +        if (cachedFetchedDataOption.isDefined) {
    +          val cachedFetchedData = cachedFetchedDataOption.get
    +          cachedFetchedData.inUse = false
    +          cachedFetchedData.lastReleasedTimestamp = System.currentTimeMillis()
    +        }
    +
    +      case None =>
    --- End diff --
    
    Yeah sometimes yes and sometimes ignorable thing, I intended to avoid the situation which end users are confused with false-negative, but I'm also open to add log message for that.
    
    Let me add some warning message, and if some reviewer stated the level or removing log, we can revisit it later.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Regarding metrics in FetchedDataPool, I just add basic metrics so that tests can leverage on verification. I was adding numActive as well as numIdle, but tracking and measuring them needs more resources, hence I'd postpone it unless someone asks me to add.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215308701
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    --- End diff --
    
    Hmm... yeah it would make sense, though it still needs to provide consumers whenever tasks request. 
    
    Two approaches come in my mind: 1. bounded & provide non-cached consumer with some delays when borrow fails 2. unbounded & apply random delay when borrowing. Former requires me to reintroduce handling of cached vs non-cached so would discard simplicity on my patch, whereas latter just requires me to add random delay but the chance to avoid resource allocation flood would be completely based on probability. (max ~50 ms might be enough?).
    
    I'd like to lean on 2 but also open to 1 if we don't feel safe with 2.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95800/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214917536
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    --- End diff --
    
    Nice catch! Will rename.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94912/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95688 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95688/testReport)** for PR 22138 at commit [`9685cc5`](https://github.com/apache/spark/commit/9685cc59255458c41d293da4117954cc653fab28).


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214797928
  
    --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedPoolSuite.scala ---
    @@ -0,0 +1,299 @@
    +/*
    + * 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.kafka.clients.consumer.ConsumerRecord
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FetchedPoolSuite extends SharedSQLContext {
    +  type Record = ConsumerRecord[Array[Byte], Array[Byte]]
    +
    +  private val dummyBytes = "dummy".getBytes
    +
    +  test("acquire fresh one") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    dataPool.release(cacheKey, data)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("acquire fetched data from multiple keys") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKeys = (0 to 10).map { partId =>
    +      CacheKey("testgroup", new TopicPartition("topic", partId))
    +    }
    +
    +    assert(dataPool.getCache.size === 0)
    +    cacheKeys.foreach { key => assert(dataPool.getCache.get(key).isEmpty) }
    +
    +    val dataList = cacheKeys.map(key => (key, dataPool.acquire(key, 0)))
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataList.map { case (_, data) =>
    +      data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +    }
    +
    +    dataList.foreach { case (key, data) =>
    +      dataPool.release(key, data)
    +    }
    +
    +    assert(dataPool.getCache.size === cacheKeys.size)
    +    cacheKeys.map { key =>
    +      assert(dataPool.getCache(key).size === 1)
    +      assert(!dataPool.getCache(key).head.inUse)
    +    }
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("continuous use of fetched data from single key") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val data = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    data.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => data.next() }
    +
    +    dataPool.release(cacheKey, data)
    +
    +    // suppose next batch
    +
    +    val data2 = dataPool.acquire(cacheKey, data.nextOffsetInFetchedData)
    +
    +    assert(data.eq(data2))
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.release(cacheKey, data2)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("multiple tasks referring same key continuously using fetched data") {
    +    val dataPool = FetchedDataPool.build
    +
    +    val cacheKey = CacheKey("testgroup", new TopicPartition("topic", 0))
    +
    +    assert(dataPool.getCache.get(cacheKey).isEmpty)
    +
    +    val dataFromTask1 = dataPool.acquire(cacheKey, 0)
    +
    +    assert(dataPool.getCache(cacheKey).size === 1)
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    val dataFromTask2 = dataPool.acquire(cacheKey, 0)
    +
    +    // it shouldn't give same object as dataFromTask1 though it asks same offset
    +    // it definitely works when offsets are not overlapped: skip adding test for that
    +    assert(dataPool.getCache(cacheKey).size === 2)
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // reading from task 1
    +    dataFromTask1.withNewPoll(testRecords(0, 5).listIterator, 5)
    +
    +    (0 to 3).foreach { _ => dataFromTask1.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask1)
    +
    +    // reading from task 2
    +    dataFromTask2.withNewPoll(testRecords(0, 30).listIterator, 30)
    +
    +    (0 to 5).foreach { _ => dataFromTask2.next() }
    +
    +    dataPool.release(cacheKey, dataFromTask2)
    +
    +    // suppose next batch for task 1
    +    val data2FromTask1 = dataPool.acquire(cacheKey, dataFromTask1.nextOffsetInFetchedData)
    +    assert(data2FromTask1.eq(dataFromTask1))
    +
    +    assert(dataPool.getCache(cacheKey).head.inUse)
    +
    +    // suppose next batch for task 2
    +    val data2FromTask2 = dataPool.acquire(cacheKey, dataFromTask2.nextOffsetInFetchedData)
    +    assert(data2FromTask2.eq(dataFromTask2))
    +
    +    assert(dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 2
    +    dataPool.release(cacheKey, data2FromTask2)
    +    assert(!dataPool.getCache(cacheKey)(1).inUse)
    +
    +    // release from task 1
    +    dataPool.release(cacheKey, data2FromTask1)
    +    assert(!dataPool.getCache(cacheKey).head.inUse)
    +
    +    dataPool.shutdown()
    +  }
    +
    +  test("evict idle fetched data") {
    --- End diff --
    
    This stays in infinite loop if `startEvictorThread` throws exception.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @gaborgsomogyi Totally makes sense. Let me address while the patch is reviewed by committers. I may get recommendations to rename the config or even more, so addressing documentation would be the last part.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    > just wondering why org.apache.spark.sql.kafka010.CachedKafkaProducer uses com.google.common.cache.LoadingCache?
    
    Because KafkaProducer is thread-safe unless it enables transaction, hence encouraged for multiple tasks to use concurrently. For consumer it is not thread-safe so we guarded with custom logic, and this patch proposes to guard with Apache Commons Pool.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213866613
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    +    // set invalidate timestamp to let active objects being destroyed when returning to pool
    +    objectFactory.keyToLastInvalidatedTimestamp.put(key, System.currentTimeMillis())
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(intConsumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(intConsumer.topicPartition, intConsumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = e match {
    +      case e1: PooledObjectInvalidated =>
    +        logDebug("Pool for key was invalidated after cached object was borrowed. " +
    +          s"Invalidating cached object - key: ${e1.key} / borrowed timestamp: " +
    +          s"${e1.lastBorrowedTime} / invalidated timestamp for key: ${e1.lastInvalidatedTimestamp}")
    +
    +      case _ => logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +      val capacity = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior of CachedInternalKafkaConsumerPool, so do not modify
    --- End diff --
    
    Yeah I changed the class name and missed to replace it with new name. Will fix.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95206 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95206/testReport)** for PR 22138 at commit [`859a9d0`](https://github.com/apache/spark/commit/859a9d0b3cb7dba7fbd727af78d9a88072501c0d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215310881
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    --- End diff --
    
    OK makes sense. Will add assertion.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95513/
    Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by attilapiros <gi...@git.apache.org>.
Github user attilapiros commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213615553
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    --- End diff --
    
    Is this call (getNumActive) really necessary?


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214911381
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    --- End diff --
    
    This is to make sure `cache` itself is not accessible from outside, and when callers access `cache` via `getCache`, they will be noted it should not be used other than testing from scaladoc.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #96295 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96295/testReport)** for PR 22138 at commit [`da5bd7f`](https://github.com/apache/spark/commit/da5bd7fd79a0ce41ca7301ecdc4e375c897e8036).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r218719952
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -18,222 +18,247 @@
     package org.apache.spark.sql.kafka010
     
     import java.{util => ju}
    +import java.io.Closeable
     import java.util.concurrent.TimeoutException
     
     import scala.collection.JavaConverters._
     
     import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException}
     import org.apache.kafka.common.TopicPartition
     
    -import org.apache.spark.{SparkEnv, SparkException, TaskContext}
    +import org.apache.spark.TaskContext
     import org.apache.spark.internal.Logging
    -import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET}
     import org.apache.spark.sql.kafka010.KafkaSourceProvider._
    -import org.apache.spark.util.UninterruptibleThread
    +import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread}
    +
    +/**
    + * This class simplifies the usages of Kafka consumer in Spark SQL Kafka connector.
    + *
    + * NOTE: Like KafkaConsumer, this class is not thread-safe.
    + * NOTE for contributors: It is possible for the instance to be used from multiple callers,
    + * so all the methods should not rely on current cursor and use seek manually.
    + */
    +private[kafka010] class InternalKafkaConsumer(
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Closeable with Logging {
    +
    +  val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +
    +  private val consumer = createConsumer
     
    -private[kafka010] sealed trait KafkaDataConsumer {
       /**
    -   * Get the record for the given offset if available.
    -   *
    -   * If the record is invisible (either a
    -   * transaction message, or an aborted message when the consumer's `isolation.level` is
    -   * `read_committed`), it will be skipped and this method will try to fetch next available record
    -   * within [offset, untilOffset).
    -   *
    -   * This method also will try its best to detect data loss. If `failOnDataLoss` is `true`, it will
    -   * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this
    -   * method will try to fetch next available record within [offset, untilOffset).
    -   *
    -   * When this method tries to skip offsets due to either invisible messages or data loss and
    -   * reaches `untilOffset`, it will return `null`.
    +   * Poll messages from Kafka starting from `offset` and returns a pair of "list of consumer record"
    +   * and "offset after poll". The list of consumer record may be empty if the Kafka consumer fetches
    +   * some messages but all of them are not visible messages (either transaction messages,
    +   * or aborted messages when `isolation.level` is `read_committed`).
        *
    -   * @param offset         the offset to fetch.
    -   * @param untilOffset    the max offset to fetch. Exclusive.
    -   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    -   * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at
    -   *                       offset if available, or throw exception.when `failOnDataLoss` is `false`,
    -   *                       this method will either return record at offset if available, or return
    -   *                       the next earliest available record less than untilOffset, or null. It
    -   *                       will not throw any exception.
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
        */
    -  def get(
    -      offset: Long,
    -      untilOffset: Long,
    -      pollTimeoutMs: Long,
    -      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -    internalConsumer.get(offset, untilOffset, pollTimeoutMs, failOnDataLoss)
    +  def fetch(offset: Long, pollTimeoutMs: Long)
    +  : (ju.List[ConsumerRecord[Array[Byte], Array[Byte]]], Long) = {
    +    // Seek to the offset because we may call seekToBeginning or seekToEnd before this.
    +    seek(offset)
    +    val p = consumer.poll(pollTimeoutMs)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
    +    val offsetAfterPoll = consumer.position(topicPartition)
    +    logDebug(s"Offset changed from $offset to $offsetAfterPoll after polling")
    +    val fetchedData = (r, offsetAfterPoll)
    +    if (r.isEmpty) {
    +      // We cannot fetch anything after `poll`. Two possible cases:
    +      // - `offset` is out of range so that Kafka returns nothing. `OffsetOutOfRangeException` will
    +      //   be thrown.
    +      // - Cannot fetch any data before timeout. `TimeoutException` will be thrown.
    +      // - Fetched something but all of them are not invisible. This is a valid case and let the
    +      //   caller handles this.
    +      val range = getAvailableOffsetRange()
    +      if (offset < range.earliest || offset >= range.latest) {
    +        throw new OffsetOutOfRangeException(
    +          Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava)
    +      } else if (offset == offsetAfterPoll) {
    +        throw new TimeoutException(
    +          s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds")
    +      }
    +    }
    +    fetchedData
       }
     
       /**
        * Return the available offset range of the current partition. It's a pair of the earliest offset
        * and the latest offset.
        */
    -  def getAvailableOffsetRange(): AvailableOffsetRange = internalConsumer.getAvailableOffsetRange()
    +  def getAvailableOffsetRange(): AvailableOffsetRange = {
    +    consumer.seekToBeginning(Set(topicPartition).asJava)
    +    val earliestOffset = consumer.position(topicPartition)
    +    consumer.seekToEnd(Set(topicPartition).asJava)
    +    val latestOffset = consumer.position(topicPartition)
    +    AvailableOffsetRange(earliestOffset, latestOffset)
    +  }
     
    -  /**
    -   * Release this consumer from being further used. Depending on its implementation,
    -   * this consumer will be either finalized, or reset for reuse later.
    -   */
    -  def release(): Unit
    +  override def close(): Unit = {
    +    consumer.close()
    +  }
     
    -  /** Reference to the internal implementation that this wrapper delegates to */
    -  protected def internalConsumer: InternalKafkaConsumer
    -}
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
    +    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
     
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $groupId $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +}
     
     /**
    - * A wrapper around Kafka's KafkaConsumer that throws error when data loss is detected.
    - * This is not for direct use outside this file.
    + * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    + *
    + * @param _records the pre-fetched Kafka records.
    + * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    + *                                 should check if the pre-fetched data is still valid.
    + * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    + *                           poll when `records` is drained.
      */
    -private[kafka010] case class InternalKafkaConsumer(
    -    topicPartition: TopicPartition,
    -    kafkaParams: ju.Map[String, Object]) extends Logging {
    -  import InternalKafkaConsumer._
    -
    -  /**
    -   * The internal object to store the fetched data from Kafka consumer and the next offset to poll.
    -   *
    -   * @param _records the pre-fetched Kafka records.
    -   * @param _nextOffsetInFetchedData the next offset in `records`. We use this to verify if we
    -   *                                 should check if the pre-fetched data is still valid.
    -   * @param _offsetAfterPoll the Kafka offset after calling `poll`. We will use this offset to
    -   *                           poll when `records` is drained.
    -   */
    -  private case class FetchedData(
    -      private var _records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -      private var _nextOffsetInFetchedData: Long,
    -      private var _offsetAfterPoll: Long) {
    -
    -    def withNewPoll(
    -        records: ju.ListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    -        offsetAfterPoll: Long): FetchedData = {
    -      this._records = records
    -      this._nextOffsetInFetchedData = UNKNOWN_OFFSET
    -      this._offsetAfterPoll = offsetAfterPoll
    -      this
    -    }
    -
    -    /** Whether there are more elements */
    -    def hasNext: Boolean = _records.hasNext
    -
    -    /** Move `records` forward and return the next record. */
    -    def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
    -      val record = _records.next()
    -      _nextOffsetInFetchedData = record.offset + 1
    -      record
    -    }
    +private[kafka010] case class FetchedData(
    --- End diff --
    
    Yeah, I think the tests are fine with minor suggestions. Thanks!


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95513 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95513/testReport)** for PR 22138 at commit [`017c0bb`](https://github.com/apache/spark/commit/017c0bbf9365b32467de64c96a1a0d6aee1f6875).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Now vote for Spark 2.4 is in progress. If we are not in stand-by mode for any blocker issues for Spark 2.4 RC, I'd be really happy if someone could revisit this and continue reviewing.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95777 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95777/testReport)** for PR 22138 at commit [`5b56bd0`](https://github.com/apache/spark/commit/5b56bd0e26e198281ebe6888a8f012efe325cdab).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213871197
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    --- End diff --
    
    That sounds good, but let's wait for voices on committers since CacheKey is designed before, not introduced in this patch.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    If you have multiple consumers for a given key, and those consumers are at different offsets, isn't it likely that the client code will not get the right consumer, leading to extra seeking?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @koeninger 
    
    I'm not sure but are you saying that an executor cares about multiple queries (multiple jobs) concurrently? I honestly didn't notice it. If that is going to be problem, we should add something (could we get query id at that time?) in cache key to differentiate consumers. If we want to avoid extra seeking due to different offsets, consumers should not be reused among with multiple queries, and that's just a matter of cache key.
    
    If you are thinking about co-use of consumers among multiple queries because of reusing connection to Kafka, I think extra seeking is unavoidable (I guess fetched data should be much more critical issue unless we never reuse after returning to pool). If seeking is light operation, we may even go with only reusing connection (not position we already sought): always resetting position (and data maybe?) when borrowing from pool or returning consumer to pool.
    
    Btw, the rationalization of this patch is not solving the issue you're referring. This patch is also based on #20767 but dealing with another improvements pointed out in comments: adopt pool library to not reinvent the wheel, and also enabling metrics regarding the pool.
    
    I'm not sure the issue you're referring is a serious one (show-stopper): if the issue is a kind of serious, someone should handle the issue once we are aware of the issue at March, or at least relevant JIRA issue should be filed with detailed explanation before. I'd like to ask you in favor of handling (or filing) the issue since you may know the issue best.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215313888
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The soft capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value,
    + * and the pool will have reasonable default value if the value is not provided.
    + * (The instance will do its best effort to respect soft capacity but it can exceed when there's
    + * a borrowing request and there's neither free space nor idle object to clear.)
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(
    +    objectFactory: ObjectFactory,
    +    poolConfig: PoolConfig) {
    +
    +  // the class is intended to have only soft capacity
    +  assert(poolConfig.getMaxTotal < 0)
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the soft capacity,
    +   * pool will try to clear some of idle objects.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +
    +    if (getTotal == poolConfig.getSoftMaxTotal()) {
    +      pool.clearOldest()
    +    }
    +
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(consumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(consumer), consumer)
    +  }
    +
    +  /** Invalidates all idle consumers for the key */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    pool.clear(key)
    +  }
    +
    +  /**
    +   * Closes the keyed object pool. Once the pool is closed,
    +   * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject
    +   * will continue to work, with returned objects destroyed on return.
    +   *
    +   * Also destroys idle instances in the pool.
    +   */
    +  def close(): Unit = {
    +    pool.close()
    +  }
    +
    +  def getNumIdle: Int = pool.getNumIdle
    +
    +  def getNumIdle(key: CacheKey): Int = pool.getNumIdle(key)
    +
    +  def getNumActive: Int = pool.getNumActive
    +
    +  def getNumActive(key: CacheKey): Int = pool.getNumActive(key)
    +
    +  def getTotal: Int = getNumIdle + getNumActive
    +
    +  def getTotal(key: CacheKey): Int = getNumIdle(key) + getNumActive(key)
    +
    +  private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = {
    +    // We can assume that kafkaParam should not be different for same cache key,
    +    // otherwise we can't reuse the cached object and cache key should contain kafkaParam.
    +    // So it should be safe to put the key/value pair only when the key doesn't exist.
    +    objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams)
    +  }
    +
    +  private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = {
    +    new CacheKey(consumer.topicPartition, consumer.kafkaParams)
    +  }
    +}
    +
    +private[kafka010] object InternalKafkaConsumerPool {
    +
    +  /**
    +   * Builds the pool for [[InternalKafkaConsumer]]. The pool instance is created per each call.
    +   */
    +  def build: InternalKafkaConsumerPool = {
    +    val objFactory = new ObjectFactory
    +    val poolConfig = new PoolConfig
    +    new InternalKafkaConsumerPool(objFactory, poolConfig)
    +  }
    +
    +  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,
    +                                     lastBorrowedTime: Long) extends RuntimeException
    +
    +  object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging {
    +    override def onSwallowException(e: Exception): Unit = {
    +      logError(s"Error closing Kafka consumer", e)
    +    }
    +  }
    +
    +  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] {
    +    private var softMaxTotal = Int.MaxValue
    +
    +    def getSoftMaxTotal(): Int = softMaxTotal
    +
    +    init()
    +
    +    def init(): Unit = {
    +      import PoolConfig._
    +
    +      val conf = SparkEnv.get.conf
    +
    +      softMaxTotal = conf.getInt(CONFIG_NAME_CAPACITY, DEFAULT_VALUE_CAPACITY)
    +
    +      val jmxEnabled = conf.getBoolean(CONFIG_NAME_JMX_ENABLED,
    +        defaultValue = DEFAULT_VALUE_JMX_ENABLED)
    +      val minEvictableIdleTimeMillis = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +        DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +      val evictorThreadRunIntervalMillis = conf.getLong(
    +        CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +        DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +      // NOTE: Below lines define the behavior, so do not modify unless you know what you are
    +      // doing, and update the class doc accordingly if necessary when you modify.
    +
    +      // 1. Set min idle objects per key to 0 to avoid creating unnecessary object.
    +      // 2. Set max idle objects per key to 3 but set total objects per key to infinite
    +      // which ensures borrowing per key is not restricted.
    +      // 3. Set max total objects to infinite which ensures all objects are managed in this pool.
    +      setMinIdlePerKey(0)
    +      setMaxIdlePerKey(3)
    +      setMaxTotalPerKey(-1)
    +      setMaxTotal(-1)
    +
    +      // Set minimum evictable idle time which will be referred from evictor thread
    +      setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis)
    +      setSoftMinEvictableIdleTimeMillis(-1)
    +
    +      // evictor thread will run test with ten idle objects
    +      setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis)
    +      setNumTestsPerEvictionRun(10)
    +      setEvictionPolicy(new DefaultEvictionPolicy[InternalKafkaConsumer]())
    +
    +      // Immediately fail on exhausted pool while borrowing
    +      setBlockWhenExhausted(false)
    +
    +      setJmxEnabled(jmxEnabled)
    +      setJmxNamePrefix("kafka010-cached-simple-kafka-consumer-pool")
    +    }
    +  }
    +
    +  object PoolConfig {
    +    val CONFIG_NAME_PREFIX = "spark.sql.kafkaConsumerCache."
    +    val CONFIG_NAME_CAPACITY = CONFIG_NAME_PREFIX + "capacity"
    +    val CONFIG_NAME_JMX_ENABLED = CONFIG_NAME_PREFIX + "jmx.enable"
    +    val CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS = CONFIG_NAME_PREFIX +
    +      "minEvictableIdleTimeMillis"
    +    val CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = CONFIG_NAME_PREFIX +
    +      "evictorThreadRunIntervalMillis"
    +
    +    val DEFAULT_VALUE_CAPACITY = 64
    +    val DEFAULT_VALUE_JMX_ENABLED = false
    +    val DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS = 5 * 60 * 1000 // 5 minutes
    +    val DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS = 3 * 60 * 1000 // 3 minutes
    +  }
    +
    +  class ObjectFactory extends BaseKeyedPooledObjectFactory[CacheKey, InternalKafkaConsumer]
    +    with Logging {
    +
    +    val keyToKafkaParams: ConcurrentHashMap[CacheKey, ju.Map[String, Object]] =
    +      new ConcurrentHashMap[CacheKey, ju.Map[String, Object]]()
    +
    +    override def create(key: CacheKey): InternalKafkaConsumer = {
    +      val kafkaParams = keyToKafkaParams.get(key)
    +      if (kafkaParams == null) {
    --- End diff --
    
    I agree it is scala idiomatic. Let me see whether it is critical path, and apply `Option` if it's not.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214716582
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    --- End diff --
    
    Why is this better then:
    `private[kafka010] val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty`


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95472/
    Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @gaborgsomogyi Yeah... I'm just waiting for it. Btw I proposed solution on SPARK-10816 as well and it is also waiting for response. I'm going to work on another item or review others so that I can avoid being blocked by Spark 2.4 RC.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98909/
    Test PASSed.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r213866495
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.ConcurrentHashMap
    +
    +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener}
    +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer._
    +
    +/**
    + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]].
    + *
    + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on
    + * the class, and same contract applies: after using the borrowed object, you must either call
    + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object
    + * should be destroyed.
    + *
    + * The capacity of pool is determined by "spark.sql.kafkaConsumerCache.capacity" config value, and
    + * the pool will have reasonable default value if the value is not provided.
    + *
    + * This class guarantees that no caller will get pooled object once the object is borrowed and
    + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]]
    + * unless caller shares the object to multiple threads.
    + */
    +private[kafka010] class InternalKafkaConsumerPool(objectFactory: ObjectFactory,
    +                                                  poolConfig: PoolConfig) {
    +
    +  private lazy val pool = {
    +    val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer](
    +      objectFactory, poolConfig)
    +    internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener)
    +    internalPool
    +  }
    +
    +  /**
    +   * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key,
    +   * the pool will create the [[InternalKafkaConsumer]] object.
    +   *
    +   * If the pool doesn't have idle object for the key and also exceeds the capacity, pool will try
    +   * to clear some of idle objects. If it doesn't help getting empty space to create new object,
    +   * it will throw [[NoSuchElementException]] immediately.
    +   *
    +   * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise
    +   * the object will be kept in pool as active object.
    +   */
    +  def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = {
    +    updateKafkaParamForKey(key, kafkaParams)
    +    pool.borrowObject(key)
    +  }
    +
    +  /** Returns borrowed object to the pool. */
    +  def returnObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.returnObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /** Invalidates (destroy) borrowed object to the pool. */
    +  def invalidateObject(intConsumer: InternalKafkaConsumer): Unit = {
    +    pool.invalidateObject(extractCacheKey(intConsumer), intConsumer)
    +  }
    +
    +  /**
    +   * Invalidates current idle and active (borrowed) objects for the key. It ensure no invalidated
    +   * object will be provided again via borrowObject.
    +   *
    +   * It doesn't mean the key will not be available: valid objects will be available via calling
    +   * borrowObject afterwards.
    +   */
    +  def invalidateKey(key: CacheKey): Unit = {
    +    // invalidate all idle consumers for the key
    +    pool.clear(key)
    +
    +    pool.getNumActive()
    --- End diff --
    
    My bad. Will remove.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #95450 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95450/testReport)** for PR 22138 at commit [`7d13ee5`](https://github.com/apache/spark/commit/7d13ee5601c6e71200700c44de067325aa057f9a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    @koeninger 
    Yeah I see what you're saying, then IMHO isolating consumers with query sounds better than others. Adding next offset to the cache key would make consumer moving bucket in cache every time it is processed, which is not expected behavior for general pool solution and we have to reinvent the wheel (and it is not ideal situation for caching, too).
    
    There's an evict thread in Apache Commons Pool running on background, and we could close consumers being idle for a long time (say 5 mins or higher). That's another benefit of adopting Apache Commons Pool (maybe available for most of general pool solutions): we could also evict cached consumers eventually which topic or partition is removed while query is running. It is not only evicted because of exceeding cache, but also time of inactivity.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r215281569
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    +
    +  private def startEvictorThread(): Unit = {
    +    executorService.scheduleAtFixedRate(new Runnable() {
    +      override def run(): Unit = {
    +        removeIdleFetchedData()
    +      }
    +    }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS)
    +  }
    +
    +  startEvictorThread()
    +
    +  def acquire(key: CacheKey, desiredStartOffset: Long): FetchedData = synchronized {
    +    val fetchedDataList = cache.getOrElseUpdate(key, new CachedFetchedDataList())
    +
    +    val cachedFetchedDataOption = fetchedDataList.find { p =>
    +      !p.inUse && p.getObject.nextOffsetInFetchedData == desiredStartOffset
    +    }
    +
    +    var cachedFetchedData: CachedFetchedData = null
    +    if (cachedFetchedDataOption.isDefined) {
    +      cachedFetchedData = cachedFetchedDataOption.get
    +    } else {
    +      cachedFetchedData = CachedFetchedData.empty()
    +      fetchedDataList += cachedFetchedData
    +    }
    +
    +    cachedFetchedData.lastAcquiredTimestamp = System.currentTimeMillis()
    +    cachedFetchedData.inUse = true
    +
    +    cachedFetchedData.getObject
    +  }
    +
    +  def invalidate(key: CacheKey): Unit = synchronized {
    +    cache.remove(key)
    +  }
    +
    +  def release(key: CacheKey, fetchedData: FetchedData): Unit = synchronized {
    +    cache.get(key) match {
    +      case Some(fetchedDataList) =>
    +        val cachedFetchedDataOption = fetchedDataList.find { p =>
    +          p.inUse && p.getObject == fetchedData
    +        }
    +
    +        if (cachedFetchedDataOption.isDefined) {
    +          val cachedFetchedData = cachedFetchedDataOption.get
    +          cachedFetchedData.inUse = false
    +          cachedFetchedData.lastReleasedTimestamp = System.currentTimeMillis()
    +        }
    +
    +      case None =>
    --- End diff --
    
    Thanks, yeah just for the record I meant something like this: https://github.com/apache/spark/blob/341b55a58964b1966a1919ac0774c8be5d5e7251/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala#L623



---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Two tests failed and not relevant to this patch.
    
    * org.apache.spark.scheduler.DAGSchedulerSuite.Barrier task failures from the same stage attempt don't trigger multiple stage retries
      * I just saw this once so would we want to file an issue for this?
    * org.apache.spark.sql.kafka010.KafkaRelationSuite.read Kafka transactional messages: read_committed
      * Patch in reviewing: #22293
    
    Will retrigger again.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94912 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94912/testReport)** for PR 22138 at commit [`c82f306`](https://github.com/apache/spark/commit/c82f3064fa8744f91b5c8a92645588dc9d53ba35).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class PooledObjectInvalidated(key: CacheKey, lastInvalidatedTimestamp: Long,`
      * `  class PoolConfig extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] `
      * `  case class CacheKey(groupId: String, topicPartition: TopicPartition) `


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    Kindly ask for reviewing. Please never mind when you're busy with fixing bugs on Spark 2.4 RC.
    
    @gaborgsomogyi  I guess I left two things for committer decision: 1. define soft boundary and log when pooled objects exceed the boundary 2. documentation. Do you have more to review?


---

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


[GitHub] spark issue #22138: [SPARK-25151][SS] Apply Apache Commons Pool to KafkaData...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22138
  
    **[Test build #94914 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94914/testReport)** for PR 22138 at commit [`fd728ef`](https://github.com/apache/spark/commit/fd728ef8c99ebb33d6dba5466e6a8dba8984248d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214716234
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.TimeUnit
    +
    +import scala.collection.mutable
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
    +import org.apache.spark.util.ThreadUtils
    +
    +/**
    + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
    + *
    + * Along with CacheKey, it receives desired start offset to find cached FetchedData which
    + * may be stored from previous batch. If it can't find one to match, it will create
    + * a new FetchedData.
    + */
    +private[kafka010] class FetchedDataPool {
    +  import FetchedDataPool._
    +
    +  private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
    +    var lastReleasedTimestamp: Long = Long.MaxValue
    +    var lastAcquiredTimestamp: Long = Long.MinValue
    +    var inUse: Boolean = false
    +
    +    def getObject: FetchedData = fetchedData
    +  }
    +
    +  private object CachedFetchedData {
    +    def empty(): CachedFetchedData = {
    +      val emptyData = FetchedData(
    +        ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
    +        UNKNOWN_OFFSET,
    +        UNKNOWN_OFFSET)
    +
    +      CachedFetchedData(emptyData)
    +    }
    +  }
    +
    +  private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
    +
    +  private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty
    +
    +  /** Retrieve internal cache. This method is only for testing. */
    +  private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache
    +
    +  private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = {
    +    val conf = SparkEnv.get.conf
    +
    +    val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS,
    +      DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS)
    +
    +    val evictorThreadInterval = conf.getLong(
    +      CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS,
    +      DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS)
    +
    +    (minEvictIdleTime, evictorThreadInterval)
    +  }
    +
    +  private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
    +    "kafka-fetched-data--cache-evictor")
    --- End diff --
    
    Nit: `kafka-fetched-data-cache-evictor`


---

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


[GitHub] spark pull request #22138: [SPARK-25151][SS] Apply Apache Commons Pool to Ka...

Posted by gaborgsomogyi <gi...@git.apache.org>.
Github user gaborgsomogyi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22138#discussion_r214803861
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala ---
    @@ -414,17 +468,37 @@ private[kafka010] case class InternalKafkaConsumer(
         }
       }
     
    -  /** Create a new consumer and reset cached states */
    -  private def resetConsumer(): Unit = {
    -    consumer.close()
    -    consumer = createConsumer
    -    fetchedData.reset()
    +  /**
    +   * Poll messages from Kafka starting from `offset` and update `fetchedData`. `fetchedData` may be
    +   * empty if the Kafka consumer fetches some messages but all of them are not visible messages
    +   * (either transaction messages, or aborted messages when `isolation.level` is `read_committed`).
    +   *
    +   * @throws OffsetOutOfRangeException if `offset` is out of range.
    +   * @throws TimeoutException if the consumer position is not changed after polling. It means the
    +   *                          consumer polls nothing before timeout.
    +   */
    +  private def fetchData(offset: Long, pollTimeoutMs: Long): Unit = {
    +    val (records, offsetAfterPoll) = consumer.fetch(offset, pollTimeoutMs)
    +    fetchedData.withNewPoll(records.listIterator, offsetAfterPoll)
    +  }
    +
    +  private def ensureConsumerAvailable(): Unit = {
    +    if (consumer == null) {
    --- End diff --
    
    Why not using option?


---

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