Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 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
4 changes: 1 addition & 3 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -714,9 +714,7 @@ private[spark] object SparkConf extends Logging {
AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0")),
KERBEROS_FILESYSTEMS_TO_ACCESS.key -> Seq(
AlternateConfig("spark.yarn.access.namenodes", "2.2"),
AlternateConfig("spark.yarn.access.hadoopFileSystems", "3.0")),
"spark.kafka.consumer.cache.capacity" -> Seq(
AlternateConfig("spark.sql.kafkaConsumerCache.capacity", "3.0"))
AlternateConfig("spark.yarn.access.hadoopFileSystems", "3.0"))
)

/**
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,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
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.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