-
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 all commits
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
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,27 @@ | ||
| /* | ||
| * 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.sql.execution.streaming.continuous.shuffle | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
|
|
||
| /** | ||
| * Trait for writing to a continuous processing shuffle. | ||
| */ | ||
| trait ContinuousShuffleWriter { | ||
| def write(epoch: Iterator[UnsafeRow]): Unit | ||
|
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. I dont think its the right interface. The
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 think it's better encapsulation to re-implement the partitioning in every ContinuousShuffleWriter implementation than to re-implement it in every ContinuousShuffleWriter user. (Note that the non-continuous ShuffleWriter has precedent for this: it uses the same interface, and all implementations of ShuffleWriter do re-implement partitioning.)
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. I see. That's fair. |
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,26 +20,24 @@ package org.apache.spark.sql.execution.streaming.continuous.shuffle | |
| import java.util.concurrent._ | ||
| import java.util.concurrent.atomic.AtomicBoolean | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} | ||
| 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 | ||
|
|
@@ -48,7 +46,7 @@ private[shuffle] case class ReceiverEpochMarker(writerId: Int) extends UnsafeRow | |
| * TODO: Support multiple source tasks. We need to output a single epoch marker once all | ||
| * source tasks have sent one. | ||
| */ | ||
| private[shuffle] class UnsafeRowReceiver( | ||
| private[shuffle] class RPCContinuousShuffleReader( | ||
|
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. nit: If we need the rename here, how about other messages name and comments?
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. Good point. Caught what I think are the rest. |
||
| queueSize: Int, | ||
| numShuffleWriters: Int, | ||
| epochIntervalMs: Long, | ||
|
|
@@ -57,7 +55,7 @@ private[shuffle] class UnsafeRowReceiver( | |
| // 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. | ||
|
|
@@ -68,7 +66,9 @@ private[shuffle] class UnsafeRowReceiver( | |
| } | ||
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
| case r: UnsafeRowReceiverMessage => | ||
| case r: RPCContinuousShuffleMessage => | ||
| // Note that this will block a thread the shared RPC handler pool! | ||
| // The TCP based shuffle handler (SPARK-24541) will avoid this problem. | ||
| queues(r.writerId).put(r) | ||
| context.reply(()) | ||
| } | ||
|
|
@@ -79,10 +79,10 @@ private[shuffle] class UnsafeRowReceiver( | |
| 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. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,60 @@ | ||
| /* | ||
| * 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.sql.execution.streaming.continuous.shuffle | ||
|
|
||
| import scala.concurrent.Future | ||
| import scala.concurrent.duration.Duration | ||
|
|
||
| import org.apache.spark.Partitioner | ||
| import org.apache.spark.rpc.RpcEndpointRef | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.util.ThreadUtils | ||
|
|
||
| /** | ||
| * A [[ContinuousShuffleWriter]] sending data to [[RPCContinuousShuffleReader]] instances. | ||
| * | ||
| * @param writerId The partition ID of this writer. | ||
| * @param outputPartitioner The partitioner on the reader side of the shuffle. | ||
| * @param endpoints The [[RPCContinuousShuffleReader]] endpoints to write to. Indexed by | ||
| * partition ID within outputPartitioner. | ||
| */ | ||
| class RPCContinuousShuffleWriter( | ||
| writerId: Int, | ||
| outputPartitioner: Partitioner, | ||
| endpoints: Array[RpcEndpointRef]) extends ContinuousShuffleWriter { | ||
|
|
||
| if (outputPartitioner.numPartitions != 1) { | ||
|
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 reason to disable it ? this should work rt?
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 believe so, but there's no way to test whether it will work until we implement the scheduling support for distributing the addresses of each of the multiple readers. |
||
| throw new IllegalArgumentException("multiple readers not yet supported") | ||
| } | ||
|
|
||
| if (outputPartitioner.numPartitions != endpoints.length) { | ||
| throw new IllegalArgumentException(s"partitioner size ${outputPartitioner.numPartitions} did " + | ||
| s"not match endpoint count ${endpoints.length}") | ||
| } | ||
|
|
||
| def write(epoch: Iterator[UnsafeRow]): Unit = { | ||
| while (epoch.hasNext) { | ||
| val row = epoch.next() | ||
| endpoints(outputPartitioner.getPartition(row)).askSync[Unit](ReceiverRow(writerId, row)) | ||
| } | ||
|
|
||
| val futures = endpoints.map(_.ask[Unit](ReceiverEpochMarker(writerId))).toSeq | ||
| implicit val ec = ThreadUtils.sameThread | ||
| ThreadUtils.awaitResult(Future.sequence(futures), Duration.Inf) | ||
| } | ||
| } | ||
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).