Skip to content
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

Wait driver endpoint start up in shuffle manger initialization #35

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,8 @@ import java.util.concurrent.{CountDownLatch, TimeUnit}
import scala.concurrent.ExecutionContext.Implicits.global
import scala.util.Success

import org.apache.spark.rpc.RpcEnv
import org.apache.spark.SparkException
import org.apache.spark.rpc.{RpcEnv, RpcEndpointRef}
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.shuffle.ucx.rpc.{UcxDriverRpcEndpoint, UcxExecutorRpcEndpoint}
import org.apache.spark.shuffle.ucx.rpc.UcxRpcMessages.{ExecutorAdded, IntroduceAllExecutors}
Expand Down Expand Up @@ -81,18 +82,28 @@ abstract class CommonUcxShuffleManager(val conf: SparkConf, isDriver: Boolean) e
val address = transport.init()
ucxTransport = transport
latch.countDown()
val rpcEnv = RpcEnv.create("ucx-rpc-env", blockManager.host, blockManager.port,
conf, new SecurityManager(conf), clientMode = false)
val rpcEnv = SparkEnv.get.rpcEnv
executorEndpoint = new UcxExecutorRpcEndpoint(rpcEnv, ucxTransport, setupThread)
val endpoint = rpcEnv.setupEndpoint(
s"ucx-shuffle-executor-${blockManager.executorId}",
executorEndpoint)
val driverEndpoint = RpcUtils.makeDriverRef(driverRpcName, conf, rpcEnv)
var driverCost = 0
var driverEndpoint: RpcEndpointRef = null
while (driverEndpoint == null) {
try {
driverEndpoint = RpcUtils.makeDriverRef(driverRpcName, conf, rpcEnv)
} catch {
case e: SparkException => {
Thread.sleep(5)
driverCost += 5
}
}
}
driverEndpoint.ask[IntroduceAllExecutors](ExecutorAdded(blockManager.executorId.toLong, endpoint,
new SerializableDirectBuffer(address)))
.andThen {
case Success(msg) =>
logInfo(s"Receive reply $msg")
logInfo(s"Driver take $driverCost ms. Receive reply ${msg.asInstanceOf[IntroduceAllExecutors].executorIdToAddress.keys}")
executorEndpoint.receive(msg)
}
}
Expand Down
Loading