-
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 24 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 | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,8 +20,6 @@ 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 | ||
|
|
@@ -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, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,54 @@ | ||
| /* | ||
| * 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.Partitioner | ||
| import org.apache.spark.rpc.RpcEndpointRef | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
|
|
||
| /** | ||
| * 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, | ||
|
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. nit: rename to
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 worry that partitionId is ambiguous with the partition to which the shuffle data is being written.
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. ok makes sense. |
||
| 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)).ask[Unit](ReceiverRow(writerId, row)) | ||
|
||
| } | ||
|
|
||
| endpoints.foreach(_.ask[Unit](ReceiverEpochMarker(writerId))) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,16 +17,36 @@ | |
|
|
||
| package org.apache.spark.sql.execution.streaming.continuous.shuffle | ||
|
|
||
| import org.apache.spark.{TaskContext, TaskContextImpl} | ||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.{HashPartitioner, Partition, TaskContext, TaskContextImpl} | ||
| import org.apache.spark.rpc.RpcEndpointRef | ||
| import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} | ||
| import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} | ||
| import org.apache.spark.sql.execution.streaming.continuous.shuffle._ | ||
| import org.apache.spark.sql.streaming.StreamTest | ||
| import org.apache.spark.sql.types.{DataType, IntegerType, StringType} | ||
| import org.apache.spark.unsafe.types.UTF8String | ||
|
|
||
| class ContinuousShuffleReadSuite extends StreamTest { | ||
| class ContinuousShuffleSuite extends StreamTest { | ||
|
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. Discussed offline. Merged these tests into the earlier test suite. Name the combined one appropriately. |
||
| // In this unit test, we emulate that we're in the task thread where | ||
| // ContinuousShuffleReadRDD.compute() will be evaluated. This requires a task context | ||
| // thread local to be set. | ||
| var ctx: TaskContextImpl = _ | ||
|
|
||
| override def beforeEach(): Unit = { | ||
| super.beforeEach() | ||
| ctx = TaskContext.empty() | ||
| TaskContext.setTaskContext(ctx) | ||
| } | ||
|
|
||
| private def unsafeRow(value: Int) = { | ||
| override def afterEach(): Unit = { | ||
| ctx.markTaskCompleted(None) | ||
| TaskContext.unset() | ||
| ctx = null | ||
| super.afterEach() | ||
| } | ||
|
|
||
| private implicit def unsafeRow(value: Int) = { | ||
|
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. Just curious: is there a reason to rearrange functions, this and below twos? Looks like they're same except changing this function to
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. And where it leverages 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. writer.write(Iterator(1, 2, 3)) and such leverages the implicit. |
||
| UnsafeProjection.create(Array(IntegerType : DataType))( | ||
| new GenericInternalRow(Array(value: Any))) | ||
| } | ||
|
|
@@ -40,22 +60,129 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| messages.foreach(endpoint.askSync[Unit](_)) | ||
| } | ||
|
|
||
| // In this unit test, we emulate that we're in the task thread where | ||
| // ContinuousShuffleReadRDD.compute() will be evaluated. This requires a task context | ||
| // thread local to be set. | ||
| var ctx: TaskContextImpl = _ | ||
| private def readRDDEndpoint(rdd: ContinuousShuffleReadRDD) = { | ||
| rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| } | ||
|
|
||
| override def beforeEach(): Unit = { | ||
| super.beforeEach() | ||
| ctx = TaskContext.empty() | ||
| TaskContext.setTaskContext(ctx) | ||
| private def readEpoch(rdd: ContinuousShuffleReadRDD) = { | ||
| rdd.compute(rdd.partitions(0), ctx).toSeq.map(_.getInt(0)) | ||
| } | ||
|
|
||
| override def afterEach(): Unit = { | ||
| ctx.markTaskCompleted(None) | ||
| TaskContext.unset() | ||
| ctx = null | ||
| super.afterEach() | ||
| test("one epoch") { | ||
|
||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new RPCContinuousShuffleWriter( | ||
| 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
|
|
||
| writer.write(Iterator(1, 2, 3)) | ||
|
|
||
| assert(readEpoch(reader) == Seq(1, 2, 3)) | ||
| } | ||
|
|
||
| test("multiple epochs") { | ||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new RPCContinuousShuffleWriter( | ||
| 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
|
|
||
| writer.write(Iterator(1, 2, 3)) | ||
| writer.write(Iterator(4, 5, 6)) | ||
|
|
||
| assert(readEpoch(reader) == Seq(1, 2, 3)) | ||
| assert(readEpoch(reader) == Seq(4, 5, 6)) | ||
| } | ||
|
|
||
| test("empty epochs") { | ||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new RPCContinuousShuffleWriter( | ||
| 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
|
|
||
| writer.write(Iterator()) | ||
| writer.write(Iterator(1, 2)) | ||
| writer.write(Iterator()) | ||
| writer.write(Iterator()) | ||
| writer.write(Iterator(3, 4)) | ||
| writer.write(Iterator()) | ||
|
|
||
| assert(readEpoch(reader) == Seq()) | ||
| assert(readEpoch(reader) == Seq(1, 2)) | ||
| assert(readEpoch(reader) == Seq()) | ||
| assert(readEpoch(reader) == Seq()) | ||
| assert(readEpoch(reader) == Seq(3, 4)) | ||
| assert(readEpoch(reader) == Seq()) | ||
| } | ||
|
|
||
| test("blocks waiting for writer") { | ||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new RPCContinuousShuffleWriter( | ||
| 0, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
|
|
||
| val readerEpoch = reader.compute(reader.partitions(0), ctx) | ||
|
|
||
| val readRowThread = new Thread { | ||
| override def run(): Unit = { | ||
| assert(readerEpoch.toSeq.map(_.getInt(0)) == Seq(1)) | ||
| } | ||
| } | ||
| readRowThread.start() | ||
|
|
||
| eventually(timeout(streamingTimeout)) { | ||
| assert(readRowThread.getState == Thread.State.TIMED_WAITING) | ||
| } | ||
|
|
||
| // Once we write the epoch the thread should stop waiting and succeed. | ||
| writer.write(Iterator(1)) | ||
| readRowThread.join() | ||
| } | ||
|
|
||
| test("multiple writer partitions") { | ||
| val numWriterPartitions = 3 | ||
|
|
||
| val reader = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) | ||
| val writers = (0 until 3).map { idx => | ||
| new RPCContinuousShuffleWriter(idx, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
| } | ||
|
|
||
| writers(0).write(Iterator(1, 4, 7)) | ||
| writers(1).write(Iterator(2, 5)) | ||
| writers(2).write(Iterator(3, 6)) | ||
|
|
||
| writers(0).write(Iterator(4, 7, 10)) | ||
| writers(1).write(Iterator(5, 8)) | ||
| writers(2).write(Iterator(6, 9)) | ||
|
|
||
| // Since there are multiple asynchronous writers, the original row sequencing is not guaranteed. | ||
| // The epochs should be deterministically preserved, however. | ||
| assert(readEpoch(reader).toSet == Seq(1, 2, 3, 4, 5, 6, 7).toSet) | ||
| assert(readEpoch(reader).toSet == Seq(4, 5, 6, 7, 8, 9, 10).toSet) | ||
| } | ||
|
|
||
| test("reader epoch only ends when all writer partitions write it") { | ||
| val numWriterPartitions = 3 | ||
|
|
||
| val reader = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) | ||
| val writers = (0 until 3).map { idx => | ||
| new RPCContinuousShuffleWriter(idx, new HashPartitioner(1), Array(readRDDEndpoint(reader))) | ||
| } | ||
|
|
||
| writers(1).write(Iterator()) | ||
| writers(2).write(Iterator()) | ||
|
|
||
| val readerEpoch = reader.compute(reader.partitions(0), ctx) | ||
|
|
||
| val readEpochMarkerThread = new Thread { | ||
| override def run(): Unit = { | ||
| assert(!readerEpoch.hasNext) | ||
| } | ||
| } | ||
|
|
||
| readEpochMarkerThread.start() | ||
| eventually(timeout(streamingTimeout)) { | ||
| assert(readEpochMarkerThread.getState == Thread.State.TIMED_WAITING) | ||
| } | ||
|
|
||
| writers(0).write(Iterator()) | ||
| readEpochMarkerThread.join() | ||
| } | ||
|
|
||
| test("receiver stopped with row last") { | ||
|
|
@@ -70,7 +197,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| ctx.markTaskCompleted(None) | ||
| val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].reader | ||
| eventually(timeout(streamingTimeout)) { | ||
| assert(receiver.asInstanceOf[UnsafeRowReceiver].stopped.get()) | ||
| assert(receiver.asInstanceOf[RPCContinuousShuffleReader].stopped.get()) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -86,11 +213,11 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| ctx.markTaskCompleted(None) | ||
| val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].reader | ||
| eventually(timeout(streamingTimeout)) { | ||
| assert(receiver.asInstanceOf[UnsafeRowReceiver].stopped.get()) | ||
| assert(receiver.asInstanceOf[RPCContinuousShuffleReader].stopped.get()) | ||
| } | ||
| } | ||
|
|
||
| test("one epoch") { | ||
| test("reader - one epoch") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| send( | ||
|
|
@@ -105,7 +232,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| assert(iter.toSeq.map(_.getInt(0)) == Seq(111, 222, 333)) | ||
| } | ||
|
|
||
| test("multiple epochs") { | ||
| test("reader - multiple epochs") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| send( | ||
|
|
@@ -124,7 +251,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| assert(secondEpoch.toSeq.map(_.getInt(0)) == Seq(222, 333)) | ||
| } | ||
|
|
||
| test("empty epochs") { | ||
| test("reader - empty epochs") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
|
|
||
|
|
@@ -148,7 +275,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| assert(rdd.compute(rdd.partitions(0), ctx).isEmpty) | ||
| } | ||
|
|
||
| test("multiple partitions") { | ||
| test("reader - multiple partitions") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 5) | ||
| // Send all data before processing to ensure there's no crossover. | ||
| for (p <- rdd.partitions) { | ||
|
|
@@ -169,7 +296,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| } | ||
| } | ||
|
|
||
| test("blocks waiting for new rows") { | ||
| test("reader - blocks waiting for new rows") { | ||
| val rdd = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, epochIntervalMs = Long.MaxValue) | ||
| val epoch = rdd.compute(rdd.partitions(0), ctx) | ||
|
|
@@ -195,7 +322,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| } | ||
| } | ||
|
|
||
| test("multiple writers") { | ||
| test("reader - multiple writers") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1, numShuffleWriters = 3) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| send( | ||
|
|
@@ -213,7 +340,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| Set("writer0-row0", "writer1-row0", "writer2-row0")) | ||
| } | ||
|
|
||
| test("epoch only ends when all writers send markers") { | ||
| test("reader - epoch only ends when all writers send markers") { | ||
| val rdd = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, numShuffleWriters = 3, epochIntervalMs = Long.MaxValue) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
|
|
@@ -233,6 +360,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
|
|
||
| // After checking the right rows, block until we get an epoch marker indicating there's no next. | ||
| // (Also fail the assertion if for some reason we get a row.) | ||
|
|
||
| val readEpochMarkerThread = new Thread { | ||
| override def run(): Unit = { | ||
| assert(!epoch.hasNext) | ||
|
|
@@ -254,7 +382,7 @@ class ContinuousShuffleReadSuite extends StreamTest { | |
| readEpochMarkerThread.join() | ||
| } | ||
|
|
||
| test("writer epochs non aligned") { | ||
| test("reader - writer epochs non aligned") { | ||
| val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1, numShuffleWriters = 3) | ||
| val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| // We send multiple epochs for 0, then multiple for 1, then multiple for 2. The receiver should | ||
|
|
||
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.
I dont think its the right interface. The
ContinuousShuffleWriterinterface should be for writing the shuffled rows. The implementation should not be responsible for actually deciding partitions (i.e.outputPartitioner.getPartition(row)), as you dont want to re-implement the partitioning in every implementation. So I think the interface should bedef write(row: UnsafeRow, partitionId: Int)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.
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.)
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.
I see. That's fair.