-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer #22138
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 6 commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
cad6a1f
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer
HeartSaVioR 8cb52e3
Address review comments
HeartSaVioR 9543745
Modify eviction UT to leverage manual scheduler and clock
HeartSaVioR 2ac46e4
Elaborate why FetchedDataPool is necessary
HeartSaVioR 25d7a52
Address review comments so far
HeartSaVioR 393eadf
Fix UTs (removing invalid test, fix missing renamed config)
HeartSaVioR 8651bf6
Reflect review comments
HeartSaVioR 7d0bf5e
Add TODOs against existing codebase - beyond the scope of current PR
HeartSaVioR 04e9ddf
Rollback my bad - they shouldn't be touched
HeartSaVioR 297f47a
Update document to reflect changes on consumer pool and fetched data …
HeartSaVioR fa12a0a
slightly modified
HeartSaVioR 74a6cbf
Address review comments, also fix the code when idle evictor interval…
HeartSaVioR 68af3d5
Reflect review comment
HeartSaVioR File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
178 changes: 178 additions & 0 deletions
178
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,178 @@ | ||
| /* | ||
| * 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(): ScheduledFuture[_] = { | ||
| executorService.scheduleAtFixedRate(() => { | ||
| Utils.tryLogNonFatalError(removeIdleFetchedData()) | ||
| }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS) | ||
| } | ||
|
|
||
| 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 | ||
| } | ||
|
|
||
| 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.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] | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.