This is an automated email from the ASF dual-hosted git repository.

joshrosen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new d53ddbe00fe [SPARK-43300][CORE] NonFateSharingCache wrapper for Guava 
Cache
d53ddbe00fe is described below

commit d53ddbe00fe73a703f870b0297278f3870148fc4
Author: Ziqi Liu <ziqi....@databricks.com>
AuthorDate: Mon May 15 18:47:29 2023 -0700

    [SPARK-43300][CORE] NonFateSharingCache wrapper for Guava Cache
    
    ### What changes were proposed in this pull request?
    
    Create `NonFateSharingCache` to wrap around Guava cache with a KeyLock to 
synchronize all requests with the same key, so they will run individually and 
fail as if they come one at a time.
    
    Wrap cache in `CodeGenerator` with `NonFateSharingCache` to protect it from 
unexpected cascade failure due to cancellation from irrelevant queries that 
loading the same key. Feel free to use this in other places where we used Guava 
cache and don't want fate-sharing behavior.
    
    Also, instead of implementing Guava Cache and LoadingCache interface, I 
define a subset of it so that we can control at compile time what cache 
operations are allowed and make sure all cache loading action go through our 
narrow waist code path with key lock. Feel free to add new APIs when needed.
    
    ### Why are the changes needed?
    
    Guava cache is widely used in spark, however, it suffers from fate-sharing 
behavior: If there are multiple requests trying to access the same key in the 
cache at the same time when the key is not in the cache, Guava cache will block 
all requests and create the object only once. If the creation fails, all 
requests will fail immediately without retry. So we might see task failure due 
to irrelevant failure in other queries due to fate sharing.
    
    This fate sharing behavior leads to unexpected results in some 
situation(for example, in code gen).
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    UT
    
    Closes #40982 from liuzqt/SPARK-43300.
    
    Authored-by: Ziqi Liu <ziqi....@databricks.com>
    Signed-off-by: Josh Rosen <joshro...@databricks.com>
---
 .../apache/spark/util/NonFateSharingCache.scala    |  78 ++++++++++++
 .../spark/util/NonFateSharingCacheSuite.scala      | 140 +++++++++++++++++++++
 .../expressions/codegen/CodeGenerator.scala        |  10 +-
 3 files changed, 225 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala 
b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala
new file mode 100644
index 00000000000..d9847313304
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.util
+
+import java.util.concurrent.Callable
+
+import com.google.common.cache.Cache
+import com.google.common.cache.LoadingCache
+
+/**
+ * SPARK-43300: Guava cache fate-sharing behavior might lead to unexpected 
cascade failure:
+ * when multiple threads access the same key in the cache at the same time 
when the key is not in
+ * the cache, Guava cache will block all requests and load the data only once. 
If the loading fails,
+ * all requests will fail immediately without retry. Therefore individual 
failure will also fail
+ * other irrelevant queries who are waiting for the same key. Given that spark 
can cancel tasks at
+ * arbitrary times for many different reasons, fate sharing means that a task 
which gets canceled
+ * while populating a cache entry can cause spurious failures in tasks from 
unrelated jobs -- even
+ * though those tasks would have successfully populated the cache if they had 
been allowed to try.
+ *
+ * This util Cache wrapper with KeyLock to synchronize threads looking for the 
same key
+ * so that they should run individually and fail as if they had arrived one at 
a time.
+ *
+ * There are so many ways to add cache entries in Guava Cache, instead of 
implementing Guava Cache
+ * and LoadingCache interface, we expose a subset of APIs so that we can 
control at compile time
+ * what cache operations are allowed.
+ */
+private[spark] object NonFateSharingCache {
+  /**
+   * This will return a NonFateSharingLoadingCache instance if user happens to 
pass a LoadingCache
+   */
+  def apply[K, V](cache: Cache[K, V]): NonFateSharingCache[K, V] = cache match 
{
+    case loadingCache: LoadingCache[K, V] => apply(loadingCache)
+    case _ => new NonFateSharingCache(cache)
+  }
+
+  def apply[K, V](loadingCache: LoadingCache[K, V]): 
NonFateSharingLoadingCache[K, V] =
+    new NonFateSharingLoadingCache(loadingCache)
+}
+
+private[spark] class NonFateSharingCache[K, V](protected val cache: Cache[K, 
V]) {
+
+  protected val keyLock = new KeyLock[K]
+
+  def get(key: K, valueLoader: Callable[_ <: V]): V = keyLock.withLock(key) {
+    cache.get(key, valueLoader)
+  }
+
+  def getIfPresent(key: Any): V = cache.getIfPresent(key)
+
+  def invalidate(key: Any): Unit = cache.invalidate(key)
+
+  def invalidateAll(): Unit = cache.invalidateAll()
+
+  def size(): Long = cache.size()
+}
+
+private[spark] class NonFateSharingLoadingCache[K, V](
+  protected val loadingCache: LoadingCache[K, V]) extends 
NonFateSharingCache[K, V](loadingCache) {
+
+  def get(key: K): V = keyLock.withLock(key) {
+    loadingCache.get(key)
+  }
+}
diff --git 
a/core/src/test/scala/org/apache/spark/util/NonFateSharingCacheSuite.scala 
b/core/src/test/scala/org/apache/spark/util/NonFateSharingCacheSuite.scala
new file mode 100644
index 00000000000..b1780e81b2c
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/NonFateSharingCacheSuite.scala
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.ExecutionException
+import java.util.concurrent.Semaphore
+import java.util.concurrent.atomic.AtomicReference
+
+import com.google.common.cache.CacheBuilder
+import com.google.common.cache.CacheLoader
+
+import org.apache.spark.SparkFunSuite
+
+object NonFateSharingCacheSuite {
+  private val TEST_KEY = "key"
+  private val FAIL_MESSAGE = "loading failed"
+  private val THREAD2_HOLDER = new AtomicReference[Thread](null)
+
+  class TestCacheLoader extends CacheLoader[String, String] {
+    var intentionalFail: ThreadLocal[Boolean] = ThreadLocal.withInitial(() => 
false)
+    var startLoading = new Semaphore(0)
+
+    def waitUntilThread2Waiting(): Unit = {
+      while (true) {
+        Thread.sleep(100)
+        if 
(Option(THREAD2_HOLDER.get()).exists(_.getState.equals(Thread.State.WAITING))) {
+          return
+        }
+      }
+    }
+
+    override def load(key: String): String = {
+      startLoading.release()
+      if (Thread.currentThread().getName.contains("test-executor1")) {
+        waitUntilThread2Waiting()
+      }
+      if (intentionalFail.get) throw new RuntimeException(FAIL_MESSAGE)
+      key
+    }
+  }
+}
+
+/**
+ * Test non-fate-sharing behavior
+ */
+class NonFateSharingCacheSuite extends SparkFunSuite {
+
+  type WorkerFunc = () => Unit
+
+  import NonFateSharingCacheSuite._
+
+  test("loading cache loading failure should not affect concurrent query on 
same key") {
+    val loader = new TestCacheLoader
+    val loadingCache: NonFateSharingLoadingCache[String, String] =
+      NonFateSharingCache(CacheBuilder.newBuilder.build(loader))
+    val thread1Task: WorkerFunc = () => {
+      loader.intentionalFail.set(true)
+      loadingCache.get(TEST_KEY)
+    }
+    val thread2Task: WorkerFunc = () => {
+      loadingCache.get(TEST_KEY)
+    }
+    testImpl(loadingCache, loader, thread1Task, thread2Task)
+  }
+
+  test("loading cache mix usage of default loader and provided loader") {
+    // Intentionally mix usage of default loader and provided value loader.
+    val loader = new TestCacheLoader
+    val loadingCache: NonFateSharingLoadingCache[String, String] =
+      NonFateSharingCache(CacheBuilder.newBuilder.build(loader))
+    val thread1Task: WorkerFunc = () => {
+      loader.intentionalFail.set(true)
+      loadingCache.get(TEST_KEY, () => loader.load(TEST_KEY)
+      )
+    }
+    val thread2Task: WorkerFunc = () => {
+      loadingCache.get(TEST_KEY)
+    }
+    testImpl(loadingCache, loader, thread1Task, thread2Task)
+  }
+
+  test("cache loading failure should not affect concurrent query on same key") 
{
+    val loader = new TestCacheLoader
+    val cache = NonFateSharingCache(CacheBuilder.newBuilder.build[String, 
String])
+    val thread1Task: WorkerFunc = () => {
+      loader.intentionalFail.set(true)
+      cache.get(
+        TEST_KEY,
+        () => loader.load(TEST_KEY)
+      )
+    }
+    val thread2Task: WorkerFunc = () => {
+      cache.get(
+        TEST_KEY,
+        () => loader.load(TEST_KEY)
+      )
+    }
+    testImpl(cache, loader, thread1Task, thread2Task)
+  }
+
+  def testImpl(
+      cache: NonFateSharingCache[String, String],
+      loader: TestCacheLoader,
+      thread1Task: WorkerFunc,
+      thread2Task: WorkerFunc): Unit = {
+    val executor1 = ThreadUtils.newDaemonSingleThreadExecutor("test-executor1")
+    val executor2 = ThreadUtils.newDaemonSingleThreadExecutor("test-executor2")
+    val r1: Runnable = () => thread1Task()
+    val r2: Runnable = () => {
+      loader.startLoading.acquire() // wait until thread1 start loading
+      THREAD2_HOLDER.set(Thread.currentThread())
+      thread2Task()
+    }
+    val f1 = executor1.submit(r1)
+    val f2 = executor2.submit(r2)
+    // thread1 should fail intentionally
+    val e = intercept[ExecutionException] {
+      f1.get
+    }
+    assert(e.getMessage.contains(FAIL_MESSAGE))
+
+    f2.get // thread 2 should not be affected by thread 1 failure
+    assert(cache.getIfPresent(TEST_KEY) != null)
+  }
+}
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 5651a30515f..fc7b2a58a17 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -46,7 +46,7 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.Platform
 import org.apache.spark.unsafe.types._
-import org.apache.spark.util.{LongAccumulator, ParentClassLoader, Utils}
+import org.apache.spark.util.{LongAccumulator, NonFateSharingCache, 
ParentClassLoader, Utils}
 
 /**
  * Java source for evaluating an [[Expression]] given a [[InternalRow]] of 
input.
@@ -1576,8 +1576,12 @@ object CodeGenerator extends Logging {
    * they are explicitly removed. A Cache on the other hand is generally 
configured to evict entries
    * automatically, in order to constrain its memory footprint.  Note that 
this cache does not use
    * weak keys/values and thus does not respond to memory pressure.
+   *
+   * Codegen can be slow. Use a non fate sharing cache in case a query gets 
canceled during codegen
+   * while other queries wait on the same code, so that those other queries 
don't get wrongly
+   * aborted. See [[NonFateSharingCache]] for more details.
    */
-  private val cache = CacheBuilder.newBuilder()
+  private val cache = NonFateSharingCache(CacheBuilder.newBuilder()
     .maximumSize(SQLConf.get.codegenCacheMaxEntries)
     .build(
       new CacheLoader[CodeAndComment, (GeneratedClass, ByteCodeStats)]() {
@@ -1593,7 +1597,7 @@ object CodeGenerator extends Logging {
           _compileTime.add(duration)
           result
         }
-      })
+      }))
 
   /**
    * Name of Java primitive data type


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to