-
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 20 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,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 org.apache.spark.{Partition, Partitioner, TaskContext} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.rpc.RpcEndpointRef | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochTracker} | ||
|
|
||
| /** | ||
| * An RDD which continuously writes epochs from its child into a continuous shuffle. | ||
| * | ||
| * @param prev The RDD to write to the continuous shuffle. | ||
| * @param outputPartitioner The partitioner on the reader side of the shuffle. | ||
| * @param endpoints The [[UnsafeRowReceiver]] endpoints to write to. Indexed by partition ID within | ||
| * outputPartitioner. | ||
| */ | ||
| class ContinuousShuffleWriteRDD( | ||
| var prev: RDD[UnsafeRow], | ||
| outputPartitioner: Partitioner, | ||
| endpoints: Seq[RpcEndpointRef]) | ||
| extends RDD[Unit](prev) { | ||
|
||
|
|
||
| override def getPartitions: Array[Partition] = prev.partitions | ||
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[Unit] = { | ||
| EpochTracker.initializeCurrentEpoch( | ||
| context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong) | ||
| val writer: ContinuousShuffleWriter = | ||
| new UnsafeRowWriter(split.index, outputPartitioner, endpoints.toArray) | ||
|
|
||
| while (!context.isInterrupted() && !context.isCompleted()) { | ||
| writer.write(prev.compute(split, context)) | ||
| EpochTracker.incrementCurrentEpoch() | ||
| } | ||
|
|
||
| Iterator() | ||
|
||
| } | ||
|
|
||
| override def clearDependencies() { | ||
| super.clearDependencies() | ||
| prev = null | ||
| } | ||
| } | ||
| 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 |
|---|---|---|
| @@ -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 [[UnsafeRowReceiver]] instances. | ||
|
||
| * | ||
| * @param writerId The partition ID of this writer. | ||
| * @param outputPartitioner The partitioner on the reader side of the shuffle. | ||
| * @param endpoints The [[UnsafeRowReceiver]] endpoints to write to. Indexed by partition ID within | ||
| * outputPartitioner. | ||
| */ | ||
| class UnsafeRowWriter( | ||
|
||
| writerId: Int, | ||
| outputPartitioner: Partitioner, | ||
| endpoints: Array[RpcEndpointRef]) extends ContinuousShuffleWriter { | ||
|
|
||
| if (outputPartitioner.numPartitions != 1) { | ||
| 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 |
|---|---|---|
| @@ -0,0 +1,205 @@ | ||
| /* | ||
| * 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.streaming.continuous.shuffle | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.{HashPartitioner, Partition, TaskContext, TaskContextImpl} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} | ||
| import org.apache.spark.sql.execution.streaming.continuous.shuffle.{ContinuousShuffleReadPartition, ContinuousShuffleReadRDD, ContinuousShuffleWriteRDD, UnsafeRowWriter} | ||
| import org.apache.spark.sql.streaming.StreamTest | ||
| import org.apache.spark.sql.types.{DataType, IntegerType} | ||
|
|
||
| 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) | ||
| } | ||
|
|
||
| override def afterEach(): Unit = { | ||
| ctx.markTaskCompleted(None) | ||
| TaskContext.unset() | ||
| ctx = null | ||
| super.afterEach() | ||
| } | ||
|
|
||
| private case class SimplePartition(index: Int) extends Partition | ||
|
|
||
| /** | ||
| * An RDD that simulates multiple continuous processing epochs, with each epoch corresponding | ||
| * to one entry in the outer epochData array. The data in the inner array is round-robined across | ||
| * the specified number of partitions. | ||
| */ | ||
| private class MultipleEpochRDD(numPartitions: Int, epochData: Array[Int]*) | ||
| extends RDD[UnsafeRow](sparkContext, Nil) { | ||
| override def getPartitions: Array[Partition] = { | ||
| (0 until numPartitions).map(SimplePartition).toArray | ||
| } | ||
|
|
||
| private val currentEpochForPartition = mutable.Map[Int, Int]().withDefaultValue(0) | ||
|
|
||
| override def compute(split: Partition, ctx: TaskContext): Iterator[UnsafeRow] = { | ||
| val epoch = epochData(currentEpochForPartition(split.index)).zipWithIndex.collect { | ||
| case (value, idx) if idx % numPartitions == split.index => unsafeRow(value) | ||
| } | ||
|
|
||
| currentEpochForPartition(split.index) += 1 | ||
| epoch.toIterator | ||
| } | ||
| } | ||
|
|
||
| private implicit def unsafeRow(value: Int) = { | ||
| UnsafeProjection.create(Array(IntegerType : DataType))( | ||
| new GenericInternalRow(Array(value: Any))) | ||
| } | ||
|
|
||
| private def readRDDEndpoint(rdd: ContinuousShuffleReadRDD) = { | ||
| rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint | ||
| } | ||
|
|
||
| private def writeEpoch(rdd: ContinuousShuffleWriteRDD, partition: Int = 0) = { | ||
| rdd.compute(rdd.partitions(partition), ctx) | ||
| } | ||
|
|
||
| private def readEpoch(rdd: ContinuousShuffleReadRDD) = { | ||
| rdd.compute(rdd.partitions(0), ctx).toSeq.map(_.getInt(0)) | ||
| } | ||
|
|
||
| test("one epoch") { | ||
|
||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new UnsafeRowWriter(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 UnsafeRowWriter(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 data = new MultipleEpochRDD(1, Array(), Array(1, 2), Array(), Array(), Array(3, 4), Array()) | ||
|
|
||
| val reader = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1) | ||
| val writer = new UnsafeRowWriter(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 UnsafeRowWriter(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") { | ||
|
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. Would we want to have another test which covers out-of-order epoch between writers (if that's valid case for us), or rely on the test in ContinuousShuffleReadRDD?
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 "reader epoch only ends when all writer partitions write it" is a sufficient test for that. |
||
| val numWriterPartitions = 3 | ||
|
|
||
| val reader = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) | ||
| val writers = (0 until 3).map { idx => | ||
| new UnsafeRowWriter(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 data = new MultipleEpochRDD(numWriterPartitions, Array()) | ||
|
|
||
| val reader = new ContinuousShuffleReadRDD( | ||
| sparkContext, numPartitions = 1, numShuffleWriters = numWriterPartitions) | ||
| val writers = (0 until 3).map { idx => | ||
| new UnsafeRowWriter(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() | ||
|
||
| } | ||
| } | ||
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.
nit: writes epoch data