sunchao commented on a change in pull request #31517:
URL: https://github.com/apache/spark/pull/31517#discussion_r676824744



##########
File path: core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
##########
@@ -84,16 +84,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)

Review comment:
       nit: why can't we just use `builder.build[Partition, 
Seq[String]](getPartitionBlockLocations)`?

##########
File path: 
core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala
##########
@@ -62,21 +61,28 @@ 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 = {

Review comment:
       nit: move this to the previous line?

##########
File path: pom.xml
##########
@@ -492,6 +492,11 @@
         <version>${guava.version}</version>
         <scope>provided</scope>
       </dependency>
+      <dependency>
+        <groupId>com.github.ben-manes.caffeine</groupId>
+        <artifactId>caffeine</artifactId>
+        <version>2.9.1</version>

Review comment:
       nit: maybe define `caffeine.version` and put it under `properties`.

##########
File path: core/src/test/scala/org/apache/spark/LocalCacheBenchmark.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import java.util.concurrent.Callable
+
+import scala.concurrent.duration.Duration
+import scala.util.Random
+
+import com.github.benmanes.caffeine.cache.{CacheLoader => CaffeineCacheLoader, 
Caffeine}
+import com.google.common.cache.{CacheBuilder, CacheLoader}
+
+import org.apache.spark.benchmark.{Benchmark, BenchmarkBase}
+import org.apache.spark.util.ThreadUtils
+
+
+
+/**
+ * Benchmark for Guava Cache vs Caffeine.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> --jars <spark core test jar>
+ *   2. build/sbt "core/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain <this 
class>"
+ *      Results will be written to "benchmarks/KryoBenchmark-results.txt".

Review comment:
       update this

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##########
@@ -159,19 +160,22 @@ class SessionCatalog(
   }
 
   private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = {
-    var builder = CacheBuilder.newBuilder()
+    var builder = Caffeine.newBuilder()
       .maximumSize(cacheSize)
 
     if (cacheTTL > 0) {
       builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS)
     }
-
-    builder.build[QualifiedTableName, LogicalPlan]()
+    // Wrapping as CaffeinatedGuava to be compatible with

Review comment:
       Hmm is this comment outdated? I don't see where `CaffeinatedGuava` is 
used.

##########
File path: core/src/test/scala/org/apache/spark/LocalCacheBenchmark.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import java.util.concurrent.Callable
+
+import scala.concurrent.duration.Duration
+import scala.util.Random
+
+import com.github.benmanes.caffeine.cache.{CacheLoader => CaffeineCacheLoader, 
Caffeine}
+import com.google.common.cache.{CacheBuilder, CacheLoader}
+
+import org.apache.spark.benchmark.{Benchmark, BenchmarkBase}
+import org.apache.spark.util.ThreadUtils
+
+

Review comment:
       nit: extra empty lines.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
##########
@@ -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 = {

Review comment:
       nit: maybe change this to:
   ```scala
     val blockManagerIdCache = Caffeine.newBuilder()
         .maximumSize(10000)
         .build[BlockManagerId, BlockManagerId](identity)
   ```
   ?

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##########
@@ -159,19 +160,22 @@ class SessionCatalog(
   }
 
   private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = {
-    var builder = CacheBuilder.newBuilder()
+    var builder = Caffeine.newBuilder()
       .maximumSize(cacheSize)
 
     if (cacheTTL > 0) {
       builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS)
     }
-
-    builder.build[QualifiedTableName, LogicalPlan]()
+    // Wrapping as CaffeinatedGuava to be compatible with
+    // the get(key, valueLoader) API of Guava cache
+    builder.build()
   }
 
   /** This method provides a way to get a cached plan. */
   def getCachedPlan(t: QualifiedTableName, c: Callable[LogicalPlan]): 
LogicalPlan = {
-    tableRelationCache.get(t, c)
+    tableRelationCache.get(t, new JFunction[QualifiedTableName, LogicalPlan] {

Review comment:
       nit: maybe:
   ```scala
       tableRelationCache.get(t, (_: QualifiedTableName) => c.call())
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to