-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34309][BUILD][CORE][SQL][K8S]Use Caffeine instead of Guava Cache #31517
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
Changes from all commits
d4e3901
a3d794a
85dc7b9
399f5b2
dbb46dc
01f99fe
18813e9
3f13689
39032e6
674af31
5f08d0f
0c5382a
4761a5b
4b49b84
1805f34
adc6d92
bdb522b
ac35393
b116f87
bd46b67
8842fa3
8985d6d
b568fcc
ca9d58d
97c3c74
f47cdb8
425c345
e0006c6
c02bc5b
c1c3cef
c9f45ae
24c361b
2b3072e
636bfb5
825b69a
d1315c3
32d9ce0
8d6ad81
cf30234
5eb75fa
7129802
045d3dc
554b5a5
f488a3b
89abb61
1a24ed4
96e6cc8
7028ffa
3fad6ef
aa742d3
95d5c2e
3ba2574
8995d72
f61b041
6146529
7c53f73
a069bff
1cbebd3
52037e1
8391e8b
0e3c5fb
4cb428d
35bea2d
f314403
994fcee
a64b280
a93e37e
56d41f1
d7610ac
93784cc
db1de65
47b0bf8
34d31fd
93330ce
e3b81c5
74e2ac3
7506999
68193a9
ee05314
db9cc56
5423f36
c6b9dc6
0743e3b
c5c84cd
f2a656a
d69df8e
9cd9c35
7b360a7
33f5353
87def1a
6c74fc6
5a75b2c
706a68d
81f863f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,12 @@ | ||
| ================================================================================================ | ||
| Loading Cache | ||
| ================================================================================================ | ||
|
|
||
| OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.7 | ||
| Intel(R) Core(TM) i5-7360U CPU @ 2.30GHz | ||
| Loading Cache: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative | ||
| -------------------------------------------------------------------------------------------------------------------------- | ||
| Guava Cache 5 6 1 15.9 62.8 1.0X | ||
| Caffeine 2 2 0 46.1 21.7 2.9X | ||
|
|
||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,12 @@ | ||
| ================================================================================================ | ||
| Loading Cache | ||
| ================================================================================================ | ||
|
|
||
| OpenJDK 64-Bit Server VM 1.8.0_232-b18 on Mac OS X 10.15.7 | ||
| Intel(R) Core(TM) i5-7360U CPU @ 2.30GHz | ||
| Loading Cache: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative | ||
| -------------------------------------------------------------------------------------------------------------------------- | ||
| Guava Cache 5 5 0 16.7 60.0 1.0X | ||
| Caffeine 2 2 1 44.3 22.6 2.7X | ||
|
|
||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,15 +18,14 @@ | |
| package org.apache.spark.deploy.history | ||
|
|
||
| import java.util.NoSuchElementException | ||
| import java.util.concurrent.ExecutionException | ||
| import java.util.concurrent.CompletionException | ||
| import javax.servlet.{DispatcherType, Filter, FilterChain, FilterConfig, ServletException, ServletRequest, ServletResponse} | ||
| import javax.servlet.http.{HttpServletRequest, HttpServletResponse} | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import com.codahale.metrics.{Counter, MetricRegistry, Timer} | ||
| import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} | ||
| import com.google.common.util.concurrent.UncheckedExecutionException | ||
| import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine, LoadingCache, RemovalCause, RemovalListener} | ||
| import org.eclipse.jetty.servlet.FilterHolder | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
|
|
@@ -62,21 +61,27 @@ private[history] class ApplicationCache( | |
|
|
||
| /** | ||
| * Removal event notifies the provider to detach the UI. | ||
| * @param rm removal notification | ||
| * @param key removal key | ||
| * @param value removal value | ||
| * @param cause the reason why a `CacheEntry` was removed, it should | ||
| * always be `SIZE` because `appCache` configured with | ||
| * `maximumSize` eviction strategy | ||
| */ | ||
| override def onRemoval(rm: RemovalNotification[CacheKey, CacheEntry]): Unit = { | ||
| override def onRemoval(key: CacheKey, value: CacheEntry, cause: RemovalCause): Unit = { | ||
| metrics.evictionCount.inc() | ||
| val key = rm.getKey | ||
| logDebug(s"Evicting entry ${key}") | ||
| operations.detachSparkUI(key.appId, key.attemptId, rm.getValue().loadedUI.ui) | ||
| logDebug(s"Evicting entry $key") | ||
| operations.detachSparkUI(key.appId, key.attemptId, value.loadedUI.ui) | ||
| } | ||
| } | ||
|
|
||
| private val appCache: LoadingCache[CacheKey, CacheEntry] = { | ||
| CacheBuilder.newBuilder() | ||
| .maximumSize(retainedApplications) | ||
| .removalListener(removalListener) | ||
| .build(appLoader) | ||
| val builder = Caffeine.newBuilder() | ||
| .maximumSize(retainedApplications) | ||
| .removalListener(removalListener) | ||
| // SPARK-34309: Use custom Executor to compatible with | ||
| // the data eviction behavior of Guava cache | ||
| .executor((command: Runnable) => command.run()) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this going to run in the same thread? Is that what the old behaviour would have been?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it's a compromise for compatibility with old behaviour at present. In the future, I will optimize this behaviour through other pr. |
||
| builder.build[CacheKey, CacheEntry](appLoader) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -86,9 +91,9 @@ private[history] class ApplicationCache( | |
|
|
||
| def get(appId: String, attemptId: Option[String] = None): CacheEntry = { | ||
| try { | ||
| appCache.get(new CacheKey(appId, attemptId)) | ||
| appCache.get(CacheKey(appId, attemptId)) | ||
| } catch { | ||
| case e @ (_: ExecutionException | _: UncheckedExecutionException) => | ||
| case e @ (_: CompletionException | _: RuntimeException) => | ||
| throw Option(e.getCause()).getOrElse(e) | ||
| } | ||
| } | ||
|
|
@@ -127,7 +132,7 @@ private[history] class ApplicationCache( | |
| } | ||
|
|
||
| /** @return Number of cached UIs. */ | ||
| def size(): Long = appCache.size() | ||
| def size(): Long = appCache.estimatedSize() | ||
|
|
||
| private def time[T](t: Timer)(f: => T): T = { | ||
| val timeCtx = t.time() | ||
|
|
@@ -197,7 +202,7 @@ private[history] class ApplicationCache( | |
| val sb = new StringBuilder(s"ApplicationCache(" + | ||
| s" retainedApplications= $retainedApplications)") | ||
| sb.append(s"; time= ${clock.getTimeMillis()}") | ||
| sb.append(s"; entry count= ${appCache.size()}\n") | ||
| sb.append(s"; entry count= ${appCache.estimatedSize()}\n") | ||
| sb.append("----\n") | ||
| appCache.asMap().asScala.foreach { | ||
| case(key, entry) => sb.append(s" $key -> $entry\n") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit | |
| import scala.reflect.ClassTag | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import com.google.common.cache.{CacheBuilder, CacheLoader} | ||
| import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine} | ||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark._ | ||
|
|
@@ -85,16 +85,18 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( | |
| } | ||
|
|
||
| // Cache of preferred locations of checkpointed files. | ||
| @transient private[spark] lazy val cachedPreferredLocations = CacheBuilder.newBuilder() | ||
| .expireAfterWrite( | ||
| SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME).get, | ||
| TimeUnit.MINUTES) | ||
| .build( | ||
| new CacheLoader[Partition, Seq[String]]() { | ||
| override def load(split: Partition): Seq[String] = { | ||
| getPartitionBlockLocations(split) | ||
| } | ||
| }) | ||
| @transient private[spark] lazy val cachedPreferredLocations = { | ||
| val builder = Caffeine.newBuilder() | ||
| .expireAfterWrite( | ||
| SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME).get, | ||
| TimeUnit.MINUTES) | ||
| val loader = new CacheLoader[Partition, Seq[String]]() { | ||
| override def load(split: Partition): Seq[String] = { | ||
| getPartitionBlockLocations(split) | ||
| } | ||
| } | ||
| builder.build[Partition, Seq[String]](loader) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: why can't we just use
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
The compile errors as follows:
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Then try to use
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 7b360a7 revert this change
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah thanks for trying it out - seems there are some issue in Java/Scala interop there. |
||
| } | ||
|
|
||
| // Returns the block locations of given partition on file system. | ||
| private def getPartitionBlockLocations(split: Partition): Seq[String] = { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.storage | |
|
|
||
| import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} | ||
|
|
||
| import com.google.common.cache.{CacheBuilder, CacheLoader} | ||
| import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine} | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.annotation.DeveloperApi | ||
|
|
@@ -136,11 +136,14 @@ private[spark] object BlockManagerId { | |
| * The max cache size is hardcoded to 10000, since the size of a BlockManagerId | ||
| * object is about 48B, the total memory cost should be below 1MB which is feasible. | ||
| */ | ||
| val blockManagerIdCache = CacheBuilder.newBuilder() | ||
| .maximumSize(10000) | ||
| .build(new CacheLoader[BlockManagerId, BlockManagerId]() { | ||
| override def load(id: BlockManagerId) = id | ||
| }) | ||
| val blockManagerIdCache = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: maybe change this to: val blockManagerIdCache = Caffeine.newBuilder()
.maximumSize(10000)
.build[BlockManagerId, BlockManagerId](identity)?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Similar to
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 7b360a7 revert this change |
||
| Caffeine.newBuilder() | ||
| .maximumSize(10000) | ||
| .build[BlockManagerId, BlockManagerId]( | ||
| new CacheLoader[BlockManagerId, BlockManagerId]() { | ||
| override def load(id: BlockManagerId): BlockManagerId = id | ||
| }) | ||
| } | ||
|
|
||
| def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = { | ||
| blockManagerIdCache.get(id) | ||
|
|
||
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.
Can you tell me why we needed to change the exception? Is this just what Caffeine throws instead? Do we have test coverage for this?
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.
com.github.benmanes.caffeine.cache.LoadingCache#getmethod throwCompletionExceptionif a checked exception was thrown while loading the value.com.google.common.cache.LoadingCache#getmethod throwExecutionExceptionif a checked exception was thrown while loading the value.So the exception type here has changed.
Uh oh!
There was an error while loading. Please reload this page.
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.
The
ExecutionException (now is CompletionException)is re-throw asRuntimeException, I need to further check whether it is covered by existing test caseThere 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.
@holdenk
testFetchWrongExecutorandtestFetchNonexistentinExternalBlockHandlerSuitealready cover this.