Github user gaborgsomogyi commented on a diff in the pull request: https://github.com/apache/spark/pull/22138#discussion_r215281569 --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/FetchedDataPool.scala --- @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import org.apache.kafka.clients.consumer.ConsumerRecord + +import org.apache.spark.SparkEnv +import org.apache.spark.sql.kafka010.KafkaDataConsumer.{CacheKey, UNKNOWN_OFFSET} +import org.apache.spark.util.ThreadUtils + +/** + * Provides object pool for [[FetchedData]] which is grouped by [[CacheKey]]. + * + * Along with CacheKey, it receives desired start offset to find cached FetchedData which + * may be stored from previous batch. If it can't find one to match, it will create + * a new FetchedData. + */ +private[kafka010] class FetchedDataPool { + import FetchedDataPool._ + + private[kafka010] case class CachedFetchedData(fetchedData: FetchedData) { + var lastReleasedTimestamp: Long = Long.MaxValue + var lastAcquiredTimestamp: Long = Long.MinValue + var inUse: Boolean = false + + def getObject: FetchedData = fetchedData + } + + private object CachedFetchedData { + def empty(): CachedFetchedData = { + val emptyData = FetchedData( + ju.Collections.emptyListIterator[ConsumerRecord[Array[Byte], Array[Byte]]], + UNKNOWN_OFFSET, + UNKNOWN_OFFSET) + + CachedFetchedData(emptyData) + } + } + + private type CachedFetchedDataList = mutable.ListBuffer[CachedFetchedData] + + private val cache: mutable.Map[CacheKey, CachedFetchedDataList] = mutable.HashMap.empty + + /** Retrieve internal cache. This method is only for testing. */ + private[kafka010] def getCache: mutable.Map[CacheKey, CachedFetchedDataList] = cache + + private val (minEvictableIdleTimeMillis, evictorThreadRunIntervalMillis): (Long, Long) = { + val conf = SparkEnv.get.conf + + val minEvictIdleTime = conf.getLong(CONFIG_NAME_MIN_EVICTABLE_IDLE_TIME_MILLIS, + DEFAULT_VALUE_MIN_EVICTABLE_IDLE_TIME_MILLIS) + + val evictorThreadInterval = conf.getLong( + CONFIG_NAME_EVICTOR_THREAD_RUN_INTERVAL_MILLIS, + DEFAULT_VALUE_EVICTOR_THREAD_RUN_INTERVAL_MILLIS) + + (minEvictIdleTime, evictorThreadInterval) + } + + private val executorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor( + "kafka-fetched-data--cache-evictor") + + private def startEvictorThread(): Unit = { + executorService.scheduleAtFixedRate(new Runnable() { + override def run(): Unit = { + removeIdleFetchedData() + } + }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS) + } + + startEvictorThread() + + def acquire(key: CacheKey, desiredStartOffset: Long): FetchedData = synchronized { + val fetchedDataList = cache.getOrElseUpdate(key, new CachedFetchedDataList()) + + val cachedFetchedDataOption = fetchedDataList.find { p => + !p.inUse && p.getObject.nextOffsetInFetchedData == desiredStartOffset + } + + var cachedFetchedData: CachedFetchedData = null + if (cachedFetchedDataOption.isDefined) { + cachedFetchedData = cachedFetchedDataOption.get + } else { + cachedFetchedData = CachedFetchedData.empty() + fetchedDataList += cachedFetchedData + } + + cachedFetchedData.lastAcquiredTimestamp = System.currentTimeMillis() + cachedFetchedData.inUse = true + + cachedFetchedData.getObject + } + + def invalidate(key: CacheKey): Unit = synchronized { + cache.remove(key) + } + + def release(key: CacheKey, fetchedData: FetchedData): Unit = synchronized { + cache.get(key) match { + case Some(fetchedDataList) => + val cachedFetchedDataOption = fetchedDataList.find { p => + p.inUse && p.getObject == fetchedData + } + + if (cachedFetchedDataOption.isDefined) { + val cachedFetchedData = cachedFetchedDataOption.get + cachedFetchedData.inUse = false + cachedFetchedData.lastReleasedTimestamp = System.currentTimeMillis() + } + + case None => --- End diff -- Thanks, yeah just for the record I meant something like this: https://github.com/apache/spark/blob/341b55a58964b1966a1919ac0774c8be5d5e7251/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala#L623
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org