-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32091][CORE] Ignore timeout error when remove blocks on the lost executor #28924
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
a09990f
8cb221e
6973d17
3053b22
5a02ef2
e5dcfcf
56ccc63
763a32c
a04a74e
5b425b5
0c6ae10
828aa2b
319d925
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 |
|---|---|---|
|
|
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer | |
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.concurrent.Future | ||
| import scala.concurrent.{Future, TimeoutException} | ||
| import scala.concurrent.duration._ | ||
| import scala.language.implicitConversions | ||
| import scala.reflect.ClassTag | ||
|
|
@@ -49,8 +49,9 @@ import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransport | |
| import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} | ||
| import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} | ||
| import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} | ||
| import org.apache.spark.rpc.RpcEnv | ||
| import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} | ||
| import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} | ||
| import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseGrainedSchedulerBackend} | ||
| import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} | ||
| import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} | ||
| import org.apache.spark.shuffle.sort.SortShuffleManager | ||
|
|
@@ -93,6 +94,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE | |
| .set(MEMORY_STORAGE_FRACTION, 0.999) | ||
| .set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") | ||
| .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) | ||
| .set(Network.RPC_ASK_TIMEOUT, "5s") | ||
|
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. Any particular reason why 5?
Member
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. In the newly added tests, we need to simulate the timeout error from BlockManager. But at the same time, we also don't want the test run too long since the default timeout value is 120s. Therefore, we choose a quite short timeout for the tests. On the other hand, we don't set it to a smaller value, e.g. 1s, which may cause test flaky. Note that the best way to set the timeout value is to set it for the newly added tests locally instead of setting it globally. However, with the limitation of the current test framework in Core side, it's hard to set it locally since it requires more changes. |
||
| } | ||
|
|
||
| private def makeBlockManager( | ||
|
|
@@ -137,8 +139,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE | |
| conf = new SparkConf(false) | ||
| init(conf) | ||
|
|
||
| rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) | ||
| rpcEnv = RpcEnv.create("test", conf.get(config.DRIVER_HOST_ADDRESS), | ||
| conf.get(config.DRIVER_PORT), conf, securityMgr) | ||
| conf.set(DRIVER_PORT, rpcEnv.address.port) | ||
| conf.set(DRIVER_HOST_ADDRESS, rpcEnv.address.host) | ||
|
|
||
| // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we | ||
| // need to create a SparkContext is to initialize LiveListenerBus. | ||
|
|
@@ -177,6 +181,105 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE | |
| blockManager.stop() | ||
| } | ||
|
|
||
| /** | ||
| * Setup driverEndpoint, executor-1(BlockManager), executor-2(BlockManager) to simulate | ||
| * the real cluster before the tests. Any requests from driver to executor-1 will be responded | ||
| * in time. However, any requests from driver to executor-2 will be timeouted, in order to test | ||
| * the specific handling of `TimeoutException`, which is raised at driver side. | ||
| * | ||
| * And, when `withLost` is true, we will not register the executor-2 to the driver. Therefore, | ||
| * it behaves like a lost executor in terms of driver's view. When `withLost` is false, we'll | ||
| * register the executor-2 normally. | ||
| */ | ||
| private def setupBlockManagerMasterWithBlocks(withLost: Boolean): Unit = { | ||
Ngone51 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // set up a simple DriverEndpoint which simply adds executorIds and | ||
| // checks whether a certain executorId has been added before. | ||
| val driverEndpoint = rpcEnv.setupEndpoint(CoarseGrainedSchedulerBackend.ENDPOINT_NAME, | ||
| new RpcEndpoint { | ||
| private val executorSet = mutable.HashSet[String]() | ||
| override val rpcEnv: RpcEnv = this.rpcEnv | ||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
| case CoarseGrainedClusterMessages.RegisterExecutor(executorId, _, _, _, _, _, _, _) => | ||
| executorSet += executorId | ||
| context.reply(true) | ||
| case CoarseGrainedClusterMessages.IsExecutorAlive(executorId) => | ||
| context.reply(executorSet.contains(executorId)) | ||
| } | ||
| } | ||
| ) | ||
|
|
||
| def createAndRegisterBlockManager(timeout: Boolean): BlockManagerId = { | ||
| val id = if (timeout) "timeout" else "normal" | ||
| val bmRef = rpcEnv.setupEndpoint(s"bm-$id", new RpcEndpoint { | ||
| override val rpcEnv: RpcEnv = this.rpcEnv | ||
| private def reply[T](context: RpcCallContext, response: T): Unit = { | ||
| if (timeout) { | ||
| Thread.sleep(conf.getTimeAsMs(Network.RPC_ASK_TIMEOUT.key) + 1000) | ||
| } | ||
| context.reply(response) | ||
| } | ||
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
| case RemoveRdd(_) => reply(context, 1) | ||
| case RemoveBroadcast(_, _) => reply(context, 1) | ||
| case RemoveShuffle(_) => reply(context, true) | ||
| } | ||
| }) | ||
| val bmId = BlockManagerId(s"exec-$id", "localhost", 1234, None) | ||
| master.registerBlockManager(bmId, Array.empty, 2000, 0, bmRef) | ||
| } | ||
|
|
||
| // set up normal bm1 | ||
| val bm1Id = createAndRegisterBlockManager(false) | ||
| // set up bm2, which intentionally takes more time than RPC_ASK_TIMEOUT to | ||
| // remove rdd/broadcast/shuffle in order to raise timeout error | ||
| val bm2Id = createAndRegisterBlockManager(true) | ||
|
|
||
| driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.RegisterExecutor( | ||
| bm1Id.executorId, null, bm1Id.host, 1, Map.empty, Map.empty, | ||
| Map.empty, 0)) | ||
|
|
||
| if (!withLost) { | ||
|
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. what does
Member
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. We always set up three roles, driver, executor-1, executor-2 before these two tests. |
||
| driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.RegisterExecutor( | ||
| bm2Id.executorId, null, bm1Id.host, 1, Map.empty, Map.empty, Map.empty, 0)) | ||
| } | ||
|
|
||
| eventually(timeout(5.seconds)) { | ||
| // make sure both bm1 and bm2 are registered at driver side BlockManagerMaster | ||
| verify(master, times(2)) | ||
| .registerBlockManager(mc.any(), mc.any(), mc.any(), mc.any(), mc.any()) | ||
| assert(driverEndpoint.askSync[Boolean]( | ||
| CoarseGrainedClusterMessages.IsExecutorAlive(bm1Id.executorId))) | ||
| assert(driverEndpoint.askSync[Boolean]( | ||
| CoarseGrainedClusterMessages.IsExecutorAlive(bm2Id.executorId)) === !withLost) | ||
| } | ||
|
|
||
| // update RDD block info for bm1 and bm2 (Broadcast and shuffle don't report block | ||
| // locations to BlockManagerMaster) | ||
| master.updateBlockInfo(bm1Id, RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100, 0) | ||
| master.updateBlockInfo(bm2Id, RDDBlockId(0, 1), StorageLevel.MEMORY_ONLY, 100, 0) | ||
| } | ||
|
|
||
| test("SPARK-32091: count failures from active executors when remove rdd/broadcast/shuffle") { | ||
| setupBlockManagerMasterWithBlocks(false) | ||
| // fail because bm2 will timeout and it's not lost anymore | ||
| assert(intercept[Exception](master.removeRdd(0, true)) | ||
| .getCause.isInstanceOf[TimeoutException]) | ||
| assert(intercept[Exception](master.removeBroadcast(0, true, true)) | ||
| .getCause.isInstanceOf[TimeoutException]) | ||
| assert(intercept[Exception](master.removeShuffle(0, true)) | ||
| .getCause.isInstanceOf[TimeoutException]) | ||
| } | ||
|
|
||
| test("SPARK-32091: ignore failures from lost executors when remove rdd/broadcast/shuffle") { | ||
| setupBlockManagerMasterWithBlocks(true) | ||
| // succeed because bm1 will remove rdd/broadcast successfully and bm2 will | ||
| // timeout but ignored as it's lost | ||
| master.removeRdd(0, true) | ||
| master.removeBroadcast(0, true, true) | ||
| master.removeShuffle(0, true) | ||
| } | ||
|
|
||
| test("StorageLevel object caching") { | ||
| val level1 = StorageLevel(false, false, false, 3) | ||
| // this should return the same object as level1 | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.