JoshRosen commented on code in PR #49212: URL: https://github.com/apache/spark/pull/49212#discussion_r1908021165
########## core/src/test/scala/org/apache/spark/util/BestEffortLazyValSuite.scala: ########## @@ -0,0 +1,99 @@ +/* + * 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.io.NotSerializableException + +import org.apache.spark.{SerializerHelper, SparkFunSuite} + +class BestEffortLazyValSuite extends SparkFunSuite with SerializerHelper { + + test("BestEffortLazy works") { + var test: Option[Object] = None + + val lazyval = new BestEffortLazyVal(() => { + test = Some(new Object()) + test + }) + + // Ensure no initialization happened before the lazy value was invoked + assert(test.isEmpty) + + // Ensure the first invocation creates a new object + assert(lazyval() == test && test.isDefined) + + // Ensure the subsequent invocation serves the same object + assert(lazyval() == test && test.isDefined) Review Comment: Here is my proposed rewrite of this test: ```diff diff --git a/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala b/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala index f0385859f14..e5fbc925a4e 100644 --- a/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala @@ -16,26 +16,47 @@ */ package org.apache.spark.util +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ + import org.apache.spark.{SerializerHelper, SparkFunSuite} class TransientBestEffortLazyValSuite extends SparkFunSuite with SerializerHelper { test("TransientBestEffortLazyVal works") { - var test: Option[Object] = None - + val numInitializerCalls = new AtomicInteger(0) + // Simulate a race condition where two threads concurrently + // initialize the lazy value: + val latch = new CountDownLatch(2) val lazyval = new TransientBestEffortLazyVal(() => { - test = Some(new Object()) - test + numInitializerCalls.incrementAndGet() + latch.countDown() + new Object() }) // Ensure no initialization happened before the lazy value was invoked - assert(test.isEmpty) + assert(numInitializerCalls.get() === 0) + + // Two threads concurrently invoke the lazy value + implicit val ec: ExecutionContext = ExecutionContext.global + val future1 = Future { lazyval() } + val future2 = Future { lazyval() } + val value1 = ThreadUtils.awaitResult(future1, 10.seconds) + val value2 = ThreadUtils.awaitResult(future2, 10.seconds) + + // The initializer should have been invoked twice (due to how we set up the + // race condition via the latch): + assert(numInitializerCalls.get() === 2) - // Ensure the first invocation creates a new object - assert(lazyval() == test && test.isDefined) + // But the value should only have been computed once: + assert(value1 eq value2) // Ensure the subsequent invocation serves the same object - assert(lazyval() == test && test.isDefined) + assert(lazyval() eq value1) + assert(numInitializerCalls.get() === 2) } test("TransientBestEffortLazyVal is serializable") { ``` I think this better test because it covers the following [mutation](https://en.wikipedia.org/wiki/Mutation_testing): ```diff diff --git a/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala b/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala index 033b783ede4..f7804174ba4 100644 --- a/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala +++ b/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala @@ -56,7 +56,7 @@ private[spark] class TransientBestEffortLazyVal[T <: AnyRef]( } else { val newValue = compute() assert(newValue != null, "compute function cannot return null.") - cached.compareAndSet(null.asInstanceOf[T], newValue) + cached.set(newValue) cached.get() } } ``` whereas the original test would pass even if the code was changed to introduce the above bug. -- 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]
