-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24235][SS] Implement continuous shuffle writer for single reader partition. #21428
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 1 commit
1d6b718
b5d1008
af40769
46456dc
2ea8a6f
955ac79
8cefb72
f91bfe7
2590292
859e6e4
b23b7bb
97f7e8f
de21b1c
7dcf51a
ad0b5aa
c9adee5
63d38d8
331f437
f3ce675
e0108d7
f400651
1aaad8d
59890d4
af1508c
65837ac
a68fae2
98d55e4
e6b9118
629455b
cb6d42b
59d6ff7
f90388c
4bbdeae
e57531d
cff37c4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,18 +26,18 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow | |
| import org.apache.spark.util.NextIterator | ||
|
|
||
| /** | ||
| * Messages for the UnsafeRowReceiver endpoint. Either an incoming row or an epoch marker. | ||
| * Messages for the RPCContinuousShuffleReader endpoint. Either an incoming row or an epoch marker. | ||
| * | ||
| * Each message comes tagged with writerId, identifying which writer the message is coming | ||
| * from. The receiver will only begin the next epoch once all writers have sent an epoch | ||
| * marker ending the current epoch. | ||
| */ | ||
| private[shuffle] sealed trait UnsafeRowReceiverMessage extends Serializable { | ||
| private[shuffle] sealed trait RPCContinuousShuffleMessage extends Serializable { | ||
| def writerId: Int | ||
| } | ||
| private[shuffle] case class ReceiverRow(writerId: Int, row: UnsafeRow) | ||
| extends UnsafeRowReceiverMessage | ||
| private[shuffle] case class ReceiverEpochMarker(writerId: Int) extends UnsafeRowReceiverMessage | ||
| extends RPCContinuousShuffleMessage | ||
| private[shuffle] case class ReceiverEpochMarker(writerId: Int) extends RPCContinuousShuffleMessage | ||
|
|
||
| /** | ||
| * RPC endpoint for receiving rows into a continuous processing shuffle task. Continuous shuffle | ||
|
|
@@ -55,7 +55,7 @@ private[shuffle] class RPCContinuousShuffleReader( | |
| // Note that this queue will be drained from the main task thread and populated in the RPC | ||
| // response thread. | ||
| private val queues = Array.fill(numShuffleWriters) { | ||
| new ArrayBlockingQueue[UnsafeRowReceiverMessage](queueSize) | ||
| new ArrayBlockingQueue[RPCContinuousShuffleMessage](queueSize) | ||
| } | ||
|
|
||
| // Exposed for testing to determine if the endpoint gets stopped on task end. | ||
|
|
@@ -66,7 +66,7 @@ private[shuffle] class RPCContinuousShuffleReader( | |
| } | ||
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
| case r: UnsafeRowReceiverMessage => | ||
| case r: RPCContinuousShuffleMessage => | ||
| queues(r.writerId).put(r) | ||
|
Member
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. This line may block PRC threads and cause some critical RPC messages delayed. In addition, if the reader fails, this line may block forever if the queue is full. I'm okey with this right now since it's an experimental feature. Could you create a SPARK ticket and add a TODO here to comment the potential issue so that we won't forget this issue?
Contributor
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. I'm not sure what a critical RPC message is in this context. This line is intended to block forever if the queue is full; the receiver should not take any action or accept any other messages until the queue stops being full.
Member
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. All RPC messages inside Spark are processed in a shared fixed thread pool, hence we cannot run blocking calls inside a RPC thread. I think we need to design a backpressure mechanism in future fundamentally because a receiver cannot block a sender sending data. For example, even if we block here, we still cannot prevent the sender sending data and they will finally fulfill the TCP buffer. We cannot just count on TCP backpressure here as we need to use the same TCP connection in order to support thousands of machines.
Contributor
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. That's a very strange characteristic for an RPC framework. I don't know what backpressure could mean other than a receiver blocking a sender from sending more data. In any case, the final shuffle mechanism isn't going to use the RPC framework, so I added a reference to it. (We can discuss in a later PR whether we want to leave this mechanism lying around or remove it once we're confident the TCP-based one is working.) |
||
| context.reply(()) | ||
| } | ||
|
|
@@ -77,10 +77,10 @@ private[shuffle] class RPCContinuousShuffleReader( | |
| private val writerEpochMarkersReceived = Array.fill(numShuffleWriters)(false) | ||
|
|
||
| private val executor = Executors.newFixedThreadPool(numShuffleWriters) | ||
| private val completion = new ExecutorCompletionService[UnsafeRowReceiverMessage](executor) | ||
| private val completion = new ExecutorCompletionService[RPCContinuousShuffleMessage](executor) | ||
|
Member
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. Are you planning to implement round-robin here? Otherwise, using an array of queries + a thread pool can be just replaced with a blocking queue.
Contributor
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. It cannot be. There's a deadlock scenario where the queue is filled with records from epoch N before all writers have sent the marker for epoch N - 1. |
||
|
|
||
| private def completionTask(writerId: Int) = new Callable[UnsafeRowReceiverMessage] { | ||
| override def call(): UnsafeRowReceiverMessage = queues(writerId).take() | ||
| private def completionTask(writerId: Int) = new Callable[RPCContinuousShuffleMessage] { | ||
| override def call(): RPCContinuousShuffleMessage = queues(writerId).take() | ||
| } | ||
|
|
||
| // Initialize by submitting tasks to read the first row from each writer. | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it possible to get the query run id here? It would be helpful to debug if the endpoint name contains the query run id and partition id.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It requires a reasonable amount of extra code. As mentioned, this is not the final shuffle mechanism (and I intend to have the TCP-based shuffle ready to go in the next Spark release).