You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by gaborgsomogyi <gi...@git.apache.org> on 2018/04/06 13:26:04 UTC

[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

GitHub user gaborgsomogyi opened a pull request:

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

    [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached consumers in CachedKafkaConsumer

    ## What changes were proposed in this pull request?
    
    `CachedKafkaConsumer` in the project kafka-0-10 is designed to maintain a pool of KafkaConsumers that can be reused. However, it was built with the assumption there will be only one thread trying to read the same Kafka TopicPartition at the same time. This assumption is not true all the time and this can inadvertently lead to ConcurrentModificationException, or worse, silent reading of incorrect data.
    
    Here is a better way to design this. The consumer pool should be smart enough to avoid concurrent use of a cached consumer. If there is another request for the same TopicPartition as a currently in-use consumer, the pool should automatically return a fresh consumer.
    
    - There are effectively two kinds of consumer that may be generated
      - Cached consumer - this should be returned to the pool at task end
      - Non-cached consumer - this should be closed at task end
    - A trait called `KafkaDataConsumer` is introduced to hide this difference from the users of the consumer so that the client code does not have to reason about whether to stop and release. They simply call `val consumer = KafkaDataConsumer.acquire` and then `consumer.release`.
    - If there is request for a consumer that is in-use, then a new consumer is generated.
    - If there is request for a consumer which is a task reattempt, then already existing cached consumers will be invalidated and a new consumer is generated. This could fix potential issues if the source of the reattempt is a malfunctioning consumer.
    - In addition, I renamed the `CachedKafkaConsumer` class to `KafkaDataConsumer` because is a misnomer given that what it returns may or may not be cached.
    
    ## How was this patch tested?
    
    A new stress test that verifies it is safe to concurrently get consumers for the same TopicPartition from the consumer pool.


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

    $ git pull https://github.com/gaborgsomogyi/spark SPARK-19185

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

    https://github.com/apache/spark/pull/20997.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 #20997
    
----
commit 0fe456b48d93ed24cc59446b79ccfb32694295bc
Author: Gabor Somogyi <ga...@...>
Date:   2018-03-20T03:04:04Z

    [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached consumers in CachedKafkaConsumer

----


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181057226
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    --- End diff --
    
    Leftover from old code, fixing.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184200402
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    --- End diff --
    
    nit: `*/`


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89274 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89274/testReport)** for PR 20997 at commit [`d776289`](https://github.com/apache/spark/commit/d776289c06d28951bfef78e9eaa81e3a464c9fc4).
     * This patch **fails to build**.
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    In the meantime found a small glitch in the SQL part. Namely if reattempt happens this line
    https://github.com/apache/spark/blob/1d758dc73b54e802fdc92be204185fe7414e6553/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala#L445
    removes the consumer from cache which will end up in this log message:
    
    ```
    13:27:07.556 INFO org.apache.spark.sql.kafka010.KafkaDataConsumer: Released a supposedly cached consumer that was not found in the cache
    ```
    
    I've solved this here by removing only the closed consumer. The marked for close will be removed in `release`.



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181063487
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    --- End diff --
    
    If this is the first attempt then the list can contain elements because other tasks can create other consumers. Imran is right the `why` was written in `KafkaRDD.scala` which is missing here. Additional explanation added.
    
    In short if the possible problematic consumer is not removed from the list then it could infect tasks run later.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89344 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89344/testReport)** for PR 20997 at commit [`215339d`](https://github.com/apache/spark/commit/215339db6f29601102465c7f16ae744236683e9e).
     * 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 pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181057829
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // Consumer exists in cache. If its in use, mark it for closing later, or close it now.
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r185451746
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    --- End diff --
    
    Removed.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89359/
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180172763
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    --- End diff --
    
    Given this, is there any advantage in passing the group ID as a parameter?


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180173206
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    --- End diff --
    
    `val topics = ju.Arrays.asList(topicPartition)`


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180280210
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    +    KafkaDataConsumer.init(16, 64, 0.75f)
    +
    +    val topic = "topic" + Random.nextInt()
    +    val data = (1 to 1000).map(_.toString)
    +    val topicPartition = new TopicPartition(topic, 0)
    +    testUtils.createTopic(topic)
    +    testUtils.sendMessages(topic, data.toArray)
    +
    +    val groupId = "groupId"
    +    val kafkaParams = Map[String, Object](
    +      GROUP_ID_CONFIG -> groupId,
    +      BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress,
    +      KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      AUTO_OFFSET_RESET_CONFIG -> "earliest",
    +      ENABLE_AUTO_COMMIT_CONFIG -> "false"
    +    )
    +
    +    val numThreads = 100
    +    val numConsumerUsages = 500
    +
    +    @volatile var error: Throwable = null
    +
    +    def consume(i: Int): Unit = {
    +      val useCache = Random.nextBoolean
    +      val taskContext = if (Random.nextBoolean) {
    +        new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null)
    +      } else {
    +        null
    +      }
    +      val consumer = KafkaDataConsumer.acquire[Array[Byte], Array[Byte]](
    +        groupId, topicPartition, kafkaParams.asJava, taskContext, useCache)
    +      try {
    +        val rcvd = 0 until data.length map { offset =>
    +          val bytes = consumer.get(offset, 10000).value()
    +          new String(bytes)
    +        }
    +        assert(rcvd == data)
    +      } catch {
    +        case e: Throwable =>
    +          error = e
    +          throw e
    +      } finally {
    +        consumer.release()
    +      }
    +    }
    +
    +    val threadPool = Executors.newFixedThreadPool(numThreads)
    +    try {
    +      val futures = (1 to numConsumerUsages).map { i =>
    +        threadPool.submit(new Runnable {
    +          override def run(): Unit = { consume(i) }
    +        })
    +      }
    +      futures.foreach(_.get(1, TimeUnit.MINUTES))
    +      assert(error == null)
    +    } finally {
    +      threadPool.shutdown()
    +    }
    +  }
    +}
    --- End diff --
    
    If this PR is intended to fix a problem with silent reading of incorrect data, can you add a test reproducing that?


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181116277
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    --- End diff --
    
    Either here having type or have to cast at `entry.getValue`. Don't see the benefit at the moment.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180805380
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    --- End diff --
    
    Changed to assign the groupId from the params.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89275/testReport)** for PR 20997 at commit [`250ad92`](https://github.com/apache/spark/commit/250ad928b6c4893c4bbf0faf1cb0a84e8c152567).


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89275/testReport)** for PR 20997 at commit [`250ad92`](https://github.com/apache/spark/commit/250ad928b6c4893c4bbf0faf1cb0a84e8c152567).
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89344 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89344/testReport)** for PR 20997 at commit [`215339d`](https://github.com/apache/spark/commit/215339db6f29601102465c7f16ae744236683e9e).


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r185451988
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180808097
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    --- End diff --
    
    Changed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180212631
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    --- End diff --
    
    any particular reason you want to use a `java.util.LinkedList`?  If you instead used a scala collection, eg. `scala.collection.mutable.ArrayBuffer`, you would avoid the `asScala`s.  Or `ListBuffer` if you actually want a `LinkedList`


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180285599
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    --- End diff --
    
    Here's another potential problem with using a linked list instead of a pool that has more smarts.
    
    Let's say I have two consumers on the same topicpartion in the same jvm.  Consumer A last read offset 1000 and pre-fetched the next N messages, consumer B last read offset 2000 and pre-fetched the next N messages.
    
    If the client code that was using consumer A last time gets consumer B out of the cache this time, the prefetch is wasted and it will have to seek.  All we're saving by caching at that point is connection time.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181076158
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    --- End diff --
    
    Is it better to copy/paste the same thing every occasion?


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181077275
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // 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()
    +          closedExistingInternalConsumers.add(existingInternalConsumer)
    +        }
    +      }
    +      existingInternalConsumers.removeAll(closedExistingInternalConsumers)
    +
    +      logDebug("Reattempt detected, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      newInternalConsumer.inUse = true
    --- End diff --
    
    Ooooh, nice catch :) Removed them.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181057345
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    --- End diff --
    
    Changed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180174985
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    --- End diff --
    
    Unnecessary comment.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184194919
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    --- End diff --
    
    nit: `private[kafka010] ` since class already has that.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #90989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90989/testReport)** for PR 20997 at commit [`6cd67c6`](https://github.com/apache/spark/commit/6cd67c6ac7b948eb791cc4871477ab0b1df4fcad).


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180174857
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    --- End diff --
    
    Same.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #90044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90044/testReport)** for PR 20997 at commit [`6cd67c6`](https://github.com/apache/spark/commit/6cd67c6ac7b948eb791cc4871477ab0b1df4fcad).


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180220465
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    --- End diff --
    
    I agree, the comment in the old code was more clear: "just in case the prior attempt failures were cache related"


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    Taken a look at the pool options I have the feeling it requires more time to come up with a proper solution. Switching back to the SQL code provided one cached consumer approach...


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180280531
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    --- End diff --
    
    This is an example of the cut & paste I was referring to.
    
    In this case, I don't believe consumer is ever reassigned, so it doesn't even need to be a var.
    
    It was reassigned in the SQL version of the code.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181066058
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    --- End diff --
    
    One can spare 3 pieces of O(1) operations on the other side would make the code less readable.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181058027
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // 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()
    +          closedExistingInternalConsumers.add(existingInternalConsumer)
    +        }
    +      }
    +      existingInternalConsumers.removeAll(closedExistingInternalConsumers)
    +
    +      logDebug("Reattempt detected, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      newInternalConsumer.inUse = true
    +      existingInternalConsumers.add(newInternalConsumer)
    +      CachedKafkaDataConsumer(newInternalConsumer)
    +
    --- End diff --
    
    Removed them.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180177328
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // Consumer exists in cache. If its in use, mark it for closing later, or close it now.
    --- End diff --
    
    it's


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    I'm fine as well.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89676 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89676/testReport)** for PR 20997 at commit [`2c45388`](https://github.com/apache/spark/commit/2c453883869921c99024c02f0a29aac395c82341).
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180283733
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    --- End diff --
    
    I'm not convinced this lazy val is clearer than simply constructing a consumer when you need it, but that's the way the SQL code is...


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181655790
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    --- End diff --
    
    That's correct, the SQL part isn't keeping a linked list pool but a single cached consumer. I was considering your suggestion and came to the same conclusion:
    
    ```
    Can you clarify why you want to allow only 1 cached consumer per topicpartition, closing any others at task end?
    
    It seems like opening and closing consumers would be less efficient than allowing a pool of more than one consumer per topicpartition.
    ```
    
    Though limiting the number of cached consumers per groupId/TopicPartition is a must as you've pointed out. On the other side if we go the SQL way it's definitely less risky. Do you think we should switch back to the `one cached consumer` approach?



---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180175754
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    --- End diff --
    
    `.foreach(_.close())`


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r185452496
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    +            logWarning(
    +              s"KafkaConsumer cache hitting max capacity of $maxCapacity, " +
    +                s"removing consumer for ${entry.getKey}")
    +               try {
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180806811
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    --- End diff --
    
    Changed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180222555
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    --- End diff --
    
    this is good, but it would be nice to have a test which checks that cached consumers are re-used when possible.  Eg this could pass just by never caching anything.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180173382
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    --- End diff --
    
    nit: break into multiple lines.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180175493
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    --- End diff --
    
    Is the full type needed here again? Almost sure the compiler can figure that out alone.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181502862
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    --- End diff --
    
    Reuse test added.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89676/
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180180433
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    --- End diff --
    
    Given that all these variables are read / updated inside `synchronized` blocks, `@volatile` is unnecessary and a little misleading.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180174651
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    --- End diff --
    
    Modifier goes in same line as class declaration.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184200701
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    --- End diff --
    
    nit: `!entry.getValue.inUse`


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #90989 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90989/testReport)** for PR 20997 at commit [`6cd67c6`](https://github.com/apache/spark/commit/6cd67c6ac7b948eb791cc4871477ab0b1df4fcad).
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180805909
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    --- End diff --
    
    Changed.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    In general, 2 things about this make me uncomfortable:
    
    - It's basically a cut-and-paste of the SQL equivalent PR, https://github.com/apache/spark/pull/20767, but it is different from both that PR and the existing DStream code.
    
    - I don't see an upper bound on the number of consumers per key, nor a way of reaping idle consumers.  If the SQL equivalent code is likely to be modified to use pooling of some kind, seems better to make a consistent decision.
    



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181647681
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    --- End diff --
    
    Yeah, makes sense. That way the problematic consumers can be thrown away but a bit slower. Being so pessimistic how it's actually implemented ends up in slower execution overall.
    
    I've just seen your comment about having only one consumer in the cache just like in the SQL code. If we go that way this question will not be relevant.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/88990/
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181072630
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    --- End diff --
    
    In general happy to hear better approach.
    
    @squito The code started with java collections and didn't want to break that. A couple of conversion was the price.
    
    @koeninger True. The last info what I've seen in the SQL area is that this approach used and discussion started how to make it better. Is there an outcome which can be applied to both area? If the situation is different please share with me.
    
    Even if `only` the connection can be enhanced that could be significant in case of SSL/TLS.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181057477
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    --- End diff --
    
    Removed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181073687
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    +    KafkaDataConsumer.init(16, 64, 0.75f)
    +
    +    val topic = "topic" + Random.nextInt()
    +    val data = (1 to 1000).map(_.toString)
    +    val topicPartition = new TopicPartition(topic, 0)
    +    testUtils.createTopic(topic)
    +    testUtils.sendMessages(topic, data.toArray)
    +
    +    val groupId = "groupId"
    +    val kafkaParams = Map[String, Object](
    +      GROUP_ID_CONFIG -> groupId,
    +      BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress,
    +      KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      AUTO_OFFSET_RESET_CONFIG -> "earliest",
    +      ENABLE_AUTO_COMMIT_CONFIG -> "false"
    +    )
    +
    +    val numThreads = 100
    +    val numConsumerUsages = 500
    +
    +    @volatile var error: Throwable = null
    +
    +    def consume(i: Int): Unit = {
    +      val useCache = Random.nextBoolean
    +      val taskContext = if (Random.nextBoolean) {
    +        new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null)
    +      } else {
    +        null
    +      }
    +      val consumer = KafkaDataConsumer.acquire[Array[Byte], Array[Byte]](
    +        groupId, topicPartition, kafkaParams.asJava, taskContext, useCache)
    +      try {
    +        val rcvd = 0 until data.length map { offset =>
    --- End diff --
    
    Changed.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    @koeninger 
    
    > I don't see an upper bound on the number of consumers per key, nor a way of reaping idle consumers. If the SQL equivalent code is likely to be modified to use pooling of some kind, seems better to make a consistent decision.
    
    When do you think the decision will happen?



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181645665
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    --- End diff --
    
    OK, leaving it how it is now.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    Do I need to do any further changes?


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #89359 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89359/testReport)** for PR 20997 at commit [`7aa3257`](https://github.com/apache/spark/commit/7aa32578950476e7d409be9ba461623e47f4714d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll `


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180175661
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    --- End diff --
    
    nit: indent more.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180221087
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    +    KafkaDataConsumer.init(16, 64, 0.75f)
    +
    +    val topic = "topic" + Random.nextInt()
    +    val data = (1 to 1000).map(_.toString)
    +    val topicPartition = new TopicPartition(topic, 0)
    +    testUtils.createTopic(topic)
    +    testUtils.sendMessages(topic, data.toArray)
    +
    +    val groupId = "groupId"
    +    val kafkaParams = Map[String, Object](
    +      GROUP_ID_CONFIG -> groupId,
    +      BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress,
    +      KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      AUTO_OFFSET_RESET_CONFIG -> "earliest",
    +      ENABLE_AUTO_COMMIT_CONFIG -> "false"
    +    )
    +
    +    val numThreads = 100
    +    val numConsumerUsages = 500
    +
    +    @volatile var error: Throwable = null
    +
    +    def consume(i: Int): Unit = {
    +      val useCache = Random.nextBoolean
    +      val taskContext = if (Random.nextBoolean) {
    +        new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null)
    +      } else {
    +        null
    +      }
    +      val consumer = KafkaDataConsumer.acquire[Array[Byte], Array[Byte]](
    +        groupId, topicPartition, kafkaParams.asJava, taskContext, useCache)
    +      try {
    +        val rcvd = 0 until data.length map { offset =>
    --- End diff --
    
    style -- just by convetion, ranges are an exception to the usual rule, they are wrapped with parens `(x until y).map`


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    I think if we can't come up with a pool design now that solves most of the
    issues, we should switch back to the one cached consumer approach that the
    SQL code is using.
    
    On Mon, Apr 16, 2018 at 3:25 AM, Gabor Somogyi <no...@github.com>
    wrote:
    
    > *@gaborgsomogyi* commented on this pull request.
    > ------------------------------
    >
    > In external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/
    > KafkaDataConsumer.scala
    > <https://github.com/apache/spark/pull/20997#discussion_r181655790>:
    >
    > > +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    > +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    > +   *
    > +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    > +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    > +   * caching them and tracking when they are in use.
    > +   */
    > +  def acquire[K, V](
    > +      groupId: String,
    > +      topicPartition: TopicPartition,
    > +      kafkaParams: ju.Map[String, Object],
    > +      context: TaskContext,
    > +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    > +    val key = new CacheKey(groupId, topicPartition)
    > +    val existingInternalConsumers = Option(cache.get(key))
    > +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    >
    > That's correct, the SQL part isn't keeping a linked list pool but a single
    > cached consumer. I was considering your suggestion and came to the same
    > conclusion:
    >
    > Can you clarify why you want to allow only 1 cached consumer per topicpartition, closing any others at task end?
    >
    > It seems like opening and closing consumers would be less efficient than allowing a pool of more than one consumer per topicpartition.
    >
    > Though limiting the number of cached consumers per groupId/TopicPartition
    > is a must as you've pointed out. On the other side if we go the SQL way
    > it's definitely less risky. Do you think we should switch back to the one
    > cached consumer approach?
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/20997#discussion_r181655790>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AAGABzOM08a0IoWTJWOi204fvKoyXc6xks5tpFWDgaJpZM4TKDOs>
    > .
    >



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184203155
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    +            logWarning(
    +              s"KafkaConsumer cache hitting max capacity of $maxCapacity, " +
    +                s"removing consumer for ${entry.getKey}")
    +               try {
    +              entry.getValue.close()
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumer = cache.get(key)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one. If prior attempt failures were cache related then this way old
    +      // problematic consumers can be removed.
    +      logDebug(s"Reattempt detected, invalidating cached consumer $existingInternalConsumer")
    +      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()
    +          // Remove the consumer from cache only if it's closed.
    +          // Marked for close consumers will be removed in release function.
    +          cache.remove(key)
    +        }
    +      }
    +
    +      logDebug("Reattempt detected, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (!useCache) {
    +      // If consumer reuse turned off, then do not use it, return a new consumer
    +      logDebug("Cache usage turned off, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (existingInternalConsumer == null) {
    +      // If consumer is not already cached, then put a new in the cache and return it
    +      logDebug("No cached consumer, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      cache.put(key, newInternalConsumer)
    +      CachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (existingInternalConsumer.inUse) {
    +      // If consumer is already cached but is currently in use, then return a new consumer
    +      logDebug("Used cached consumer found, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else {
    +      // If consumer is already cached and is currently not in use, then return that consumer
    +      logDebug(s"Not used cached consumer found, re-using it $existingInternalConsumer")
    +      existingInternalConsumer.inUse = true
    +      // Any given TopicPartition should have a consistent key and value type
    +      CachedKafkaDataConsumer(existingInternalConsumer.asInstanceOf[InternalKafkaConsumer[K, V]])
    +    }
    +  }
    +
    +  private def release(internalConsumer: InternalKafkaConsumer[_, _]): Unit = synchronized {
    --- End diff --
    
    After reading this code and also the `acquire` method, is there a useful difference between the `CachedKafkaDataConsumer` and `NonCachedKafkaDataConsumer` types?
    
    It seems like the code doesn't really care about those types, but just about whether the consumer is in the cache?


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184200766
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    +            logWarning(
    +              s"KafkaConsumer cache hitting max capacity of $maxCapacity, " +
    +                s"removing consumer for ${entry.getKey}")
    +               try {
    --- End diff --
    
    nit: indent


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #90044 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90044/testReport)** for PR 20997 at commit [`6cd67c6`](https://github.com/apache/spark/commit/6cd67c6ac7b948eb791cc4871477ab0b1df4fcad).
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    I'm fine with it. Unless Cody beats me to it or has more comments, I'll push this after the long weekend.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180172573
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    --- End diff --
    
    nit: follow coding style for multi-line declarations.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180178237
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // 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()
    +          closedExistingInternalConsumers.add(existingInternalConsumer)
    +        }
    +      }
    +      existingInternalConsumers.removeAll(closedExistingInternalConsumers)
    +
    +      logDebug("Reattempt detected, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      newInternalConsumer.inUse = true
    +      existingInternalConsumers.add(newInternalConsumer)
    +      CachedKafkaDataConsumer(newInternalConsumer)
    +
    --- End diff --
    
    All these blank lines before the closing brace are unnecessary.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181066695
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    --- End diff --
    
    Yeah, it's an overkill. Removed volatile and switched to val.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180174808
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    --- End diff --
    
    Multiple lines or drop the braces.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181506582
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    --- End diff --
    
    The problem isn't that you're invalidating all N consumers, it's potentially invalidating N + 1 + 2 + ... (N -1), right?
    
    I'm saying I think you can solve this by, if it's a retry, and you would normally grab 1 consumer from the cache, invalidate just that one and make a fresh one.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r184210716
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer[_, _]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +            entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer[_, _]]): Boolean = {
    +
    +          // Try to remove the least-used entry if its currently not in use.
    +          //
    +          // If you cannot remove it, then the cache will keep growing. In the worst case,
    +          // the cache will grow to the max number of concurrent tasks that can run in the executor,
    +          // (that is, number of tasks slots) after which it will never reduce. This is unlikely to
    +          // be a serious problem because an executor with more than 64 (default) tasks slots is
    +          // likely running on a beefy machine that can handle a large number of simultaneously
    +          // active consumers.
    +
    +          if (entry.getValue.inUse == false && this.size > maxCapacity) {
    +            logWarning(
    +              s"KafkaConsumer cache hitting max capacity of $maxCapacity, " +
    +                s"removing consumer for ${entry.getKey}")
    +               try {
    +              entry.getValue.close()
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumer = cache.get(key)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one. If prior attempt failures were cache related then this way old
    +      // problematic consumers can be removed.
    +      logDebug(s"Reattempt detected, invalidating cached consumer $existingInternalConsumer")
    +      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()
    +          // Remove the consumer from cache only if it's closed.
    +          // Marked for close consumers will be removed in release function.
    +          cache.remove(key)
    +        }
    +      }
    +
    +      logDebug("Reattempt detected, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (!useCache) {
    +      // If consumer reuse turned off, then do not use it, return a new consumer
    +      logDebug("Cache usage turned off, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (existingInternalConsumer == null) {
    +      // If consumer is not already cached, then put a new in the cache and return it
    +      logDebug("No cached consumer, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      cache.put(key, newInternalConsumer)
    +      CachedKafkaDataConsumer(newInternalConsumer)
    +    } else if (existingInternalConsumer.inUse) {
    +      // If consumer is already cached but is currently in use, then return a new consumer
    +      logDebug("Used cached consumer found, new non-cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      NonCachedKafkaDataConsumer(newInternalConsumer)
    +    } else {
    +      // If consumer is already cached and is currently not in use, then return that consumer
    +      logDebug(s"Not used cached consumer found, re-using it $existingInternalConsumer")
    +      existingInternalConsumer.inUse = true
    +      // Any given TopicPartition should have a consistent key and value type
    +      CachedKafkaDataConsumer(existingInternalConsumer.asInstanceOf[InternalKafkaConsumer[K, V]])
    +    }
    +  }
    +
    +  private def release(internalConsumer: InternalKafkaConsumer[_, _]): Unit = synchronized {
    --- End diff --
    
    I think that's a good observation.  But I'm not sure it's worth deviating from the same design being used in the SQL code.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180284812
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    --- End diff --
    
    Is invalidating all consumers for a topicpartition on a re-attempt unnecessarily pessimistic?  Seems like if you have N consumers for the same topicpartition and they're all involved in a re-attempt, the last one to go through this section is going to invalidate all the fresh consumers the others just made.
    
    I think all you need is to make sure you get a fresh consumer on a re-attempt.



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180780119
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    --- End diff --
    
    Leftover from old code, fixing.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r185451919
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,359 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  private[kafka010] def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010] class InternalKafkaConsumer[K, V](
    +    val topicPartition: TopicPartition,
    +    val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  private[kafka010] val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)
    +    .asInstanceOf[String]
    +
    +  private val consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val topics = ju.Arrays.asList(topicPartition)
    +    c.assign(topics)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) {
    +      poll(timeout)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010] case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +private[kafka010] object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse)
    +    override def release(): Unit = KafkaDataConsumer.release(internalConsumer)
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = internalConsumer.close()
    +  }
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private[kafka010] var cache: ju.Map[CacheKey, InternalKafkaConsumer[_, _]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180806244
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    --- End diff --
    
    Leftover from old code, fixing.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180174410
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    --- End diff --
    
    This generally means the class should be `private` not `private[blah]`.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180810282
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    --- End diff --
    
    Removed.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181507520
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    --- End diff --
    
    The SQL code, at least last time I looked at it, isn't keeping a linked list pool, it's just making a fresh consumer that's marked for close, right?  I actually think that's better as a stopgap solution, because it's less likely to leak in some unforeseen way, and it's consistent across both codebases.
    
    
    



---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180282891
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    --- End diff --
    
    This is mostly vestigial (there used to be a remove method that took a groupId, but no kafkaParams, so there was symmetry).  
    
    I don't see a reason it can't be changed to match the SQL version at this point, i.e. assign groupId from the params.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181074560
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    +    KafkaDataConsumer.init(16, 64, 0.75f)
    +
    +    val topic = "topic" + Random.nextInt()
    +    val data = (1 to 1000).map(_.toString)
    +    val topicPartition = new TopicPartition(topic, 0)
    +    testUtils.createTopic(topic)
    +    testUtils.sendMessages(topic, data.toArray)
    +
    +    val groupId = "groupId"
    +    val kafkaParams = Map[String, Object](
    +      GROUP_ID_CONFIG -> groupId,
    +      BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress,
    +      KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      AUTO_OFFSET_RESET_CONFIG -> "earliest",
    +      ENABLE_AUTO_COMMIT_CONFIG -> "false"
    +    )
    +
    +    val numThreads = 100
    +    val numConsumerUsages = 500
    +
    +    @volatile var error: Throwable = null
    +
    +    def consume(i: Int): Unit = {
    +      val useCache = Random.nextBoolean
    +      val taskContext = if (Random.nextBoolean) {
    +        new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null)
    +      } else {
    +        null
    +      }
    +      val consumer = KafkaDataConsumer.acquire[Array[Byte], Array[Byte]](
    +        groupId, topicPartition, kafkaParams.asJava, taskContext, useCache)
    +      try {
    +        val rcvd = 0 until data.length map { offset =>
    +          val bytes = consumer.get(offset, 10000).value()
    +          new String(bytes)
    +        }
    +        assert(rcvd == data)
    +      } catch {
    +        case e: Throwable =>
    +          error = e
    +          throw e
    +      } finally {
    +        consumer.release()
    +      }
    +    }
    +
    +    val threadPool = Executors.newFixedThreadPool(numThreads)
    +    try {
    +      val futures = (1 to numConsumerUsages).map { i =>
    +        threadPool.submit(new Runnable {
    +          override def run(): Unit = { consume(i) }
    +        })
    +      }
    +      futures.foreach(_.get(1, TimeUnit.MINUTES))
    +      assert(error == null)
    +    } finally {
    +      threadPool.shutdown()
    +    }
    +  }
    +}
    --- End diff --
    
    That's a baad cut and paste issue. This PR intends to solve ` ConcurrentModificationException`.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180176691
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    --- End diff --
    
    This goes back to that assert somewhere else in the code. Given the existence of that assert, having this as a separate parameter seems confusing and unnecessary.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180283864
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    +        // 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()
    +          closedExistingInternalConsumers.add(existingInternalConsumer)
    +        }
    +      }
    +      existingInternalConsumers.removeAll(closedExistingInternalConsumers)
    +
    +      logDebug("Reattempt detected, new cached consumer will be allocated " +
    +        s"$newInternalConsumer")
    +      newInternalConsumer.inUse = true
    --- End diff --
    
    For all of these lines that set newInternalConsumer.inUse, is there any way it wouldn't already have been true?


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180808081
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    --- End diff --
    
    Changed.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #88990 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88990/testReport)** for PR 20997 at commit [`0fe456b`](https://github.com/apache/spark/commit/0fe456b48d93ed24cc59446b79ccfb32694295bc).
     * 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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181506863
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    --- End diff --
    
    Copy paste a one line static constructor only in the cases a new consumer actually needs to be constructed?  Yes, I think that's clearer than a lazy val.  But again, probably not worth the difference from the SQL code.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180179223
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    --- End diff --
    
    I'm a little confused why this needs special treatment.
    
    If this is the first attempt, won't the list just be empty? And then you could execute the same code and it would basically be a no-op?
    
    Or, from a different angle, why can't you reuse the consumers? Isn't the problem just concurrent use? So if the consumer is not in use, it should be fair game, right?


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    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 #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181084444
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    +      .getOrElse(new ju.LinkedList[InternalKafkaConsumer[_, _]])
    +
    +    cache.putIfAbsent(key, existingInternalConsumers)
    +
    +    lazy val newInternalConsumer = new InternalKafkaConsumer[K, V](
    +      groupId, topicPartition, kafkaParams)
    +
    +    if (context != null && context.attemptNumber >= 1) {
    +      // If this is reattempt at running the task, then invalidate cached consumers if any and
    +      // start with a new one.
    +      logDebug("Reattempt detected, invalidating cached consumers")
    +      val closedExistingInternalConsumers = new ju.LinkedList[InternalKafkaConsumer[_, _]]()
    +      existingInternalConsumers.asScala.foreach { existingInternalConsumer =>
    --- End diff --
    
    I was thinking about this and maybe came to the wrong conclusion.
    
    I was thinking that executors are really long living in streaming. If that's true then a problematic consumer would stay there for long time poisoning all the further tasks running there. Giving back a new consumer purely lead to this situation. What have I missed?


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180807486
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    --- End diff --
    
    Leftover from old code, fixing.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    That being the case, merging to master.


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

    https://github.com/apache/spark/pull/20997
  
    **[Test build #88990 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88990/testReport)** for PR 20997 at commit [`0fe456b`](https://github.com/apache/spark/commit/0fe456b48d93ed24cc59446b79ccfb32694295bc).


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181098163
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    +    KafkaDataConsumer.init(16, 64, 0.75f)
    +
    +    val topic = "topic" + Random.nextInt()
    +    val data = (1 to 1000).map(_.toString)
    +    val topicPartition = new TopicPartition(topic, 0)
    +    testUtils.createTopic(topic)
    +    testUtils.sendMessages(topic, data.toArray)
    +
    +    val groupId = "groupId"
    +    val kafkaParams = Map[String, Object](
    +      GROUP_ID_CONFIG -> groupId,
    +      BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress,
    +      KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName,
    +      AUTO_OFFSET_RESET_CONFIG -> "earliest",
    +      ENABLE_AUTO_COMMIT_CONFIG -> "false"
    +    )
    +
    +    val numThreads = 100
    +    val numConsumerUsages = 500
    +
    +    @volatile var error: Throwable = null
    +
    +    def consume(i: Int): Unit = {
    +      val useCache = Random.nextBoolean
    +      val taskContext = if (Random.nextBoolean) {
    +        new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null)
    +      } else {
    +        null
    +      }
    +      val consumer = KafkaDataConsumer.acquire[Array[Byte], Array[Byte]](
    +        groupId, topicPartition, kafkaParams.asJava, taskContext, useCache)
    +      try {
    +        val rcvd = 0 until data.length map { offset =>
    +          val bytes = consumer.get(offset, 10000).value()
    +          new String(bytes)
    +        }
    +        assert(rcvd == data)
    +      } catch {
    +        case e: Throwable =>
    +          error = e
    +          throw e
    +      } finally {
    +        consumer.release()
    +      }
    +    }
    +
    +    val threadPool = Executors.newFixedThreadPool(numThreads)
    +    try {
    +      val futures = (1 to numConsumerUsages).map { i =>
    +        threadPool.submit(new Runnable {
    +          override def run(): Unit = { consume(i) }
    +        })
    +      }
    +      futures.foreach(_.get(1, TimeUnit.MINUTES))
    +      assert(error == null)
    +    } finally {
    +      threadPool.shutdown()
    +    }
    +  }
    +}
    --- End diff --
    
    Removed from the PR description.


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r180180011
  
    --- Diff: external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala ---
    @@ -0,0 +1,381 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.{util => ju}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
    +import org.apache.kafka.common.{KafkaException, TopicPartition}
    +
    +import org.apache.spark.TaskContext
    +import org.apache.spark.internal.Logging
    +
    +private[kafka010] sealed trait KafkaDataConsumer[K, V] {
    +  /**
    +   * Get the record for the given offset if available.
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.get(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   *
    +   * @param offset         the offset to fetch.
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    internalConsumer.compactedStart(offset, pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   *
    +   * @param pollTimeoutMs  timeout in milliseconds to poll data from Kafka.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedNext(pollTimeoutMs)
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   *
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    internalConsumer.compactedPrevious()
    +  }
    +
    +  /**
    +   * 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
    +
    +  /** Reference to the internal implementation that this wrapper delegates to */
    +  protected def internalConsumer: InternalKafkaConsumer[K, V]
    +}
    +
    +
    +/**
    + * A wrapper around Kafka's KafkaConsumer.
    + * This is not for direct use outside this file.
    + */
    +private[kafka010]
    +class InternalKafkaConsumer[K, V](
    +  val groupId: String,
    +  val topicPartition: TopicPartition,
    +  val kafkaParams: ju.Map[String, Object]) extends Logging {
    +
    +  require(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG),
    +    "groupId used for cache key must match the groupId in kafkaParams")
    +
    +  @volatile private var consumer = createConsumer
    +
    +  /** indicates whether this consumer is in use or not */
    +  @volatile var inUse = true
    +
    +  /** indicate whether this consumer is going to be stopped in the next release */
    +  @volatile var markedForClose = false
    +
    +  // TODO if the buffer was kept around as a random-access structure,
    +  // could possibly optimize re-calculating of an RDD in the same batch
    +  @volatile private var buffer = ju.Collections.emptyListIterator[ConsumerRecord[K, V]]()
    +  @volatile private var nextOffset = InternalKafkaConsumer.UNKNOWN_OFFSET
    +
    +  override def toString: String = {
    +    "InternalKafkaConsumer(" +
    +      s"hash=${Integer.toHexString(hashCode)}, " +
    +      s"groupId=$groupId, " +
    +      s"topicPartition=$topicPartition)"
    +  }
    +
    +  /** Create a KafkaConsumer to fetch records for `topicPartition` */
    +  private def createConsumer: KafkaConsumer[K, V] = {
    +    val c = new KafkaConsumer[K, V](kafkaParams)
    +    val tps = new ju.ArrayList[TopicPartition]()
    +    tps.add(topicPartition)
    +    c.assign(tps)
    +    c
    +  }
    +
    +  def close(): Unit = consumer.close()
    +
    +  /**
    +   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
    +   * Sequential forward access will use buffers, but random access will be horribly inefficient.
    +   */
    +  def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = {
    +    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffset requested $offset")
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +    }
    +
    +    if (!buffer.hasNext()) { poll(timeout) }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +    var record = buffer.next()
    +
    +    if (record.offset != offset) {
    +      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(timeout)
    +      require(buffer.hasNext(),
    +        s"Failed to get records for $groupId $topicPartition $offset after polling for $timeout")
    +      record = buffer.next()
    +      require(record.offset == offset,
    +        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset " +
    +          s"got offset ${record.offset} instead. If this is a compacted topic, consider enabling " +
    +          "spark.streaming.kafka.allowNonConsecutiveOffsets"
    +      )
    +    }
    +
    +    nextOffset = offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Start a batch on a compacted topic
    +   */
    +  def compactedStart(offset: Long, pollTimeoutMs: Long): Unit = {
    +    logDebug(s"compacted start $groupId $topicPartition starting $offset")
    +    // This seek may not be necessary, but it's hard to tell due to gaps in compacted topics
    +    if (offset != nextOffset) {
    +      logInfo(s"Initial fetch for compacted $groupId $topicPartition $offset")
    +      seek(offset)
    +      poll(pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Get the next record in the batch from a compacted topic.
    +   * Assumes compactedStart has been called first, and ignores gaps.
    +   */
    +  def compactedNext(pollTimeoutMs: Long): ConsumerRecord[K, V] = {
    +    if (!buffer.hasNext()) {
    +      poll(pollTimeoutMs)
    +    }
    +    require(buffer.hasNext(),
    +      s"Failed to get records for compacted $groupId $topicPartition " +
    +        s"after polling for $pollTimeoutMs")
    +    val record = buffer.next()
    +    nextOffset = record.offset + 1
    +    record
    +  }
    +
    +  /**
    +   * Rewind to previous record in the batch from a compacted topic.
    +   * @throws NoSuchElementException if no previous element
    +   */
    +  def compactedPrevious(): ConsumerRecord[K, V] = {
    +    buffer.previous()
    +  }
    +
    +  private def seek(offset: Long): Unit = {
    +    logDebug(s"Seeking to $topicPartition $offset")
    +    consumer.seek(topicPartition, offset)
    +  }
    +
    +  private def poll(timeout: Long): Unit = {
    +    val p = consumer.poll(timeout)
    +    val r = p.records(topicPartition)
    +    logDebug(s"Polled ${p.partitions()}  ${r.size}")
    +    buffer = r.listIterator
    +  }
    +
    +}
    +
    +private[kafka010]
    +object KafkaDataConsumer extends Logging {
    +
    +  private case class CachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    assert(internalConsumer.inUse) // make sure this has been set to true
    +    override def release(): Unit = { KafkaDataConsumer.release(internalConsumer) }
    +  }
    +
    +  private case class NonCachedKafkaDataConsumer[K, V](internalConsumer: InternalKafkaConsumer[K, V])
    +    extends KafkaDataConsumer[K, V] {
    +    override def release(): Unit = { internalConsumer.close() }
    +  }
    +
    +  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
    +
    +  // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap
    +  private var cache: ju.Map[CacheKey, ju.List[InternalKafkaConsumer[_, _]]] = null
    +
    +  /**
    +   * Must be called before acquire, once per JVM, to configure the cache.
    +   * Further calls are ignored.
    +   * */
    +  def init(
    +      initialCapacity: Int,
    +      maxCapacity: Int,
    +      loadFactor: Float): Unit = synchronized {
    +    if (null == cache) {
    +      logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor")
    +      cache = new ju.LinkedHashMap[CacheKey, ju.List[InternalKafkaConsumer[_, _]]](
    +        initialCapacity, loadFactor, true) {
    +        override def removeEldestEntry(
    +          entry: ju.Map.Entry[CacheKey, ju.List[InternalKafkaConsumer[_, _]]]): Boolean = {
    +          if (this.size > maxCapacity) {
    +            try {
    +              entry.getValue.asScala.foreach { _.close() }
    +            } catch {
    +              case x: KafkaException =>
    +                logError("Error closing oldest Kafka consumer", x)
    +            }
    +            true
    +          } else {
    +            false
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Get a cached consumer for groupId, assigned to topic and partition.
    +   * If matching consumer doesn't already exist, will be created using kafkaParams.
    +   * The returned consumer must be released explicitly using [[KafkaDataConsumer.release()]].
    +   *
    +   * Note: This method guarantees that the consumer returned is not currently in use by anyone
    +   * else. Within this guarantee, this method will make a best effort attempt to re-use consumers by
    +   * caching them and tracking when they are in use.
    +   */
    +  def acquire[K, V](
    +      groupId: String,
    +      topicPartition: TopicPartition,
    +      kafkaParams: ju.Map[String, Object],
    +      context: TaskContext,
    +      useCache: Boolean): KafkaDataConsumer[K, V] = synchronized {
    +    val key = new CacheKey(groupId, topicPartition)
    +    val existingInternalConsumers = Option(cache.get(key))
    --- End diff --
    
    Shouldn't you just skip messing with the cache if `useCache = false`?


---

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


[GitHub] spark issue #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of cached ...

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

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


---

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


[GitHub] spark pull request #20997: [SPARK-19185] [DSTREAMS] Avoid concurrent use of ...

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

    https://github.com/apache/spark/pull/20997#discussion_r181109231
  
    --- Diff: external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.streaming.kafka010
    +
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.util.Random
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig._
    +import org.apache.kafka.common.TopicPartition
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark._
    +
    +class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var testUtils: KafkaTestUtils = _
    +
    +  override def beforeAll {
    +    super.beforeAll()
    +    testUtils = new KafkaTestUtils
    +    testUtils.setup()
    +  }
    +
    +  override def afterAll {
    +    if (testUtils != null) {
    +      testUtils.teardown()
    +      testUtils = null
    +    }
    +    super.afterAll()
    +  }
    +
    +  test("concurrent use of KafkaDataConsumer") {
    --- End diff --
    
    Reuse test added.


---

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