Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
73 changes: 64 additions & 9 deletions docs/structured-streaming-kafka-integration.md
Original file line number Diff line number Diff line change
Expand Up @@ -430,20 +430,75 @@ The following configurations are optional:
### Consumer Caching

It's time-consuming to initialize Kafka consumers, especially in streaming scenarios where processing time is a key factor.
Because of this, Spark caches Kafka consumers on executors. The caching key is built up from the following information:
Because of this, Spark pools Kafka consumers on executors, by leveraging Apache Commons Pool.

The caching key is built up from the following information:

* Topic name
* Topic partition
* Group ID

The size of the cache is limited by <code>spark.kafka.consumer.cache.capacity</code> (default: 64).
If this threshold is reached, it tries to remove the least-used entry that is currently not in use.
If it cannot be removed, 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.
The following properties are available to configure the consumer pool:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a reason why I don't see spark.kafka.consumer.cache.evictorThreadRunInterval here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same missing. Will fix.


<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td>spark.kafka.consumer.cache.capacity</td>
<td>The maximum number of consumers cached. Please note that it's a soft limit.</td>
<td>64</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.timeout</td>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Super nit: The actual timeout + evictorThreadRunInterval combination may end-up in ~6 minutes total timeout. This shouldn't be a blocker but maybe it would be better to use evictorThreadRunInterval=1m. Same applies on the other place.

Copy link
Contributor Author

@HeartSaVioR HeartSaVioR Sep 4, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense. Evictor thread doesn't check all idle objects (for Apache Commons Pool) so OK to have shorter interval.

<td>The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor.</td>
<td>5m (5 minutes)</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.evictorThreadRunInterval</td>
<td>The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run.</td>
<td>1m (1 minutes)</td>
</tr>
<tr>
<td>spark.kafka.consumer.cache.jmx.enable</td>
<td>Enable or disable JMX for pools created with this configuration instance. Statistics of the pool are available via JMX instance.
The prefix of JMX name is set to "kafka010-cached-simple-kafka-consumer-pool".
</td>
<td>false</td>
</tr>
</table>

The size of the pool is limited by <code>spark.kafka.consumer.cache.capacity</code>,
but it works as "soft-limit" to not block Spark tasks.

Idle eviction thread periodically removes consumers which are not used longer than given timeout.
If this threshold is reached when borrowing, it tries to remove the least-used entry that is currently not in use.

If it cannot be removed, then the pool will keep growing. In the worst case, the pool will grow to
the max number of concurrent tasks that can run in the executor (that is, number of task slots).

If a task fails for any reason, the new task is executed with a newly created Kafka consumer for safety reasons.
At the same time, we invalidate all consumers in pool which have same caching key, to remove consumer which was used
in failed execution. Consumers which any other tasks are using will not be closed, but will be invalidated as well
when they are returned into pool.

Along with consumers, Spark pools the records fetched from Kafka separately, to let Kafka consumers stateless in point
of Spark's view, and maximize the efficiency of pooling. It leverages same cache key with Kafka consumers pool.
Note that it doesn't leverage Apache Commons Pool due to the difference of characteristics.

The following properties are available to configure the fetched data pool:

If a task fails for any reason the new task is executed with a newly created Kafka consumer for safety reasons.
At the same time the cached Kafka consumer which was used in the failed execution will be invalidated. Here it has to
be emphasized it will not be closed if any other task is using it.
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td>spark.kafka.consumer.fetchedData.cache.timeout</td>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, not sure if I understand it. It's a timeout but at the same time number of fetched data cached. So how should I interpret this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My bad - just a copy-and-paste error. Will fix all missing things.

<td>The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor.</td>
<td>5m (5 minutes)</td>
</tr>
<tr>
<td>spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval</td>
<td>The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run.</td>
<td>1m (1 minutes)</td>
</tr>
</table>

## Writing Data to Kafka

Expand Down
10 changes: 10 additions & 0 deletions external/kafka-0-10-sql/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,11 @@
<artifactId>kafka-clients</artifactId>
<version>${kafka.version}</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-pool2</artifactId>
<version>${commons-pool2.version}</version>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${scala.binary.version}</artifactId>
Expand Down Expand Up @@ -125,6 +130,11 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-tags_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.jmock</groupId>
<artifactId>jmock-junit4</artifactId>
<scope>test</scope>
</dependency>

<!--
This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,183 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.kafka010

import java.{util => ju}
import java.util.concurrent.{ScheduledExecutorService, ScheduledFuture, TimeUnit}
import java.util.concurrent.atomic.LongAdder

import scala.collection.mutable

import org.apache.kafka.clients.consumer.ConsumerRecord

import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET}
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}

/**
* Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]].
*
* Along with CacheKey, it receives desired start offset to find cached FetchedData which
* may be stored from previous batch. If it can't find one to match, it will create
* a new FetchedData. As "desired start offset" plays as second level of key which can be
* modified in same instance, this class cannot be replaced with general pool implementations
* including Apache Commons Pool which pools KafkaConsumer.
*/
private[kafka010] class FetchedDataPool(
executorService: ScheduledExecutorService,
clock: Clock,
conf: SparkConf) extends Logging {
import FetchedDataPool._

def this(sparkConf: SparkConf) = {
this(
ThreadUtils.newDaemonSingleThreadScheduledExecutor(
"kafka-fetched-data-cache-evictor"), new SystemClock, sparkConf)
}

private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty

private val minEvictableIdleTimeMillis = conf.get(FETCHED_DATA_CACHE_TIMEOUT)
private val evictorThreadRunIntervalMillis =
conf.get(FETCHED_DATA_CACHE_EVICTOR_THREAD_RUN_INTERVAL)

private def startEvictorThread(): Option[ScheduledFuture[_]] = {
if (evictorThreadRunIntervalMillis > 0) {
val future = executorService.scheduleAtFixedRate(() => {
Utils.tryLogNonFatalError(removeIdleFetchedData())
}, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS)
Some(future)
} else {
None
}
}

private var scheduled = startEvictorThread()

private val numCreatedFetchedData = new LongAdder()
private val numTotalElements = new LongAdder()

def numCreated: Long = numCreatedFetchedData.sum()
def numTotal: Long = numTotalElements.sum()

def acquire(key: CacheKey, desiredStartOffset: Long): FetchedData = synchronized {
val fetchedDataList = cache.getOrElseUpdate(key, new CachedFetchedDataList())

val cachedFetchedDataOption = fetchedDataList.find { p =>
!p.inUse && p.getObject.nextOffsetInFetchedData == desiredStartOffset
}

var cachedFetchedData: CachedFetchedData = null
if (cachedFetchedDataOption.isDefined) {
cachedFetchedData = cachedFetchedDataOption.get
} else {
cachedFetchedData = CachedFetchedData.empty()
fetchedDataList += cachedFetchedData

numCreatedFetchedData.increment()
numTotalElements.increment()
}

cachedFetchedData.lastAcquiredTimestamp = clock.getTimeMillis()
cachedFetchedData.inUse = true

cachedFetchedData.getObject
}

def invalidate(key: CacheKey): Unit = synchronized {
cache.remove(key) match {
case Some(lst) => numTotalElements.add(-1 * lst.size)
case None =>
}
}

def release(key: CacheKey, fetchedData: FetchedData): Unit = synchronized {
def warnReleasedDataNotInPool(key: CacheKey, fetchedData: FetchedData): Unit = {
logWarning(s"No matching data in pool for $fetchedData in key $key. " +
"It might be released before, or it was not a part of pool.")
}

cache.get(key) match {
case Some(fetchedDataList) =>
val cachedFetchedDataOption = fetchedDataList.find { p =>
p.inUse && p.getObject == fetchedData
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it an expected case to try to release an idle entry, or one that is not known? I'm wondering whether there should be an assert or at least a log message, since as far as I can tell that would mean some code is trying to reuse a cached object after it's been released.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. Agreed it should be assertion or warn log message placed there. Let's add assertion and throw exception as commons-pool does it for now.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm... that should be same level of warning whether key itself doesn't exist or data is not in pool for the key. I'll deal with logWarning for now.

}

if (cachedFetchedDataOption.isEmpty) {
warnReleasedDataNotInPool(key, fetchedData)
} else {
val cachedFetchedData = cachedFetchedDataOption.get
cachedFetchedData.inUse = false
cachedFetchedData.lastReleasedTimestamp = clock.getTimeMillis()
}

case None =>
warnReleasedDataNotInPool(key, fetchedData)
}
}

def shutdown(): Unit = {
ThreadUtils.shutdown(executorService)
}

def reset(): Unit = synchronized {
scheduled.foreach(_.cancel(true))

cache.clear()
numTotalElements.reset()
numCreatedFetchedData.reset()

scheduled = startEvictorThread()
}

private def removeIdleFetchedData(): Unit = synchronized {
val now = clock.getTimeMillis()
val maxAllowedReleasedTimestamp = now - minEvictableIdleTimeMillis
cache.values.foreach { p: CachedFetchedDataList =>
val expired = p.filter { q =>
!q.inUse && q.lastReleasedTimestamp < maxAllowedReleasedTimestamp
}
p --= expired
numTotalElements.add(-1 * expired.size)
}
}
}

private[kafka010] object FetchedDataPool {
private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) {
var lastReleasedTimestamp: Long = Long.MaxValue
var lastAcquiredTimestamp: Long = Long.MinValue
var inUse: Boolean = false

def getObject: FetchedData = fetchedData
}

private object CachedFetchedData {
def empty(): CachedFetchedData = {
val emptyData = FetchedData(
ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]],
UNKNOWN_OFFSET,
UNKNOWN_OFFSET)

CachedFetchedData(emptyData)
}
}

private[kafka010] type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData]
}
Loading