Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
35 commits
Select commit Hold shift + click to select a range
1d6b718
continuous shuffle read RDD
jose-torres May 15, 2018
b5d1008
docs
jose-torres May 17, 2018
af40769
Merge remote-tracking branch 'apache/master' into readerRddMaster
jose-torres May 17, 2018
46456dc
fix ctor
jose-torres May 17, 2018
2ea8a6f
multiple partition test
jose-torres May 17, 2018
955ac79
unset task context after test
jose-torres May 17, 2018
8cefb72
conf from RDD
jose-torres May 18, 2018
f91bfe7
endpoint name
jose-torres May 18, 2018
2590292
testing bool
jose-torres May 18, 2018
859e6e4
tests
jose-torres May 18, 2018
b23b7bb
take instead of poll
jose-torres May 18, 2018
97f7e8f
add interface
jose-torres May 18, 2018
de21b1c
clarify comment
jose-torres May 18, 2018
7dcf51a
multiple
jose-torres May 18, 2018
ad0b5aa
writer with 1 reader partition
jose-torres May 25, 2018
c9adee5
docs and iface
jose-torres May 25, 2018
63d38d8
Merge remote-tracking branch 'apache/master' into writerTask
jose-torres May 25, 2018
331f437
increment epoch
jose-torres May 25, 2018
f3ce675
undo oop
jose-torres May 25, 2018
e0108d7
make rdd loop
jose-torres May 25, 2018
f400651
remote write RDD
jose-torres May 25, 2018
1aaad8d
rename classes
jose-torres May 25, 2018
59890d4
combine suites
jose-torres May 25, 2018
af1508c
fully rm old suite
jose-torres May 25, 2018
65837ac
reorder tests
jose-torres May 29, 2018
a68fae2
return future
jose-torres May 31, 2018
98d55e4
finish getting rid of old name
jose-torres May 31, 2018
e6b9118
synchronous
jose-torres May 31, 2018
629455b
finish rename
jose-torres May 31, 2018
cb6d42b
add timeouts
jose-torres Jun 13, 2018
59d6ff7
unalign
jose-torres Jun 13, 2018
f90388c
add note
jose-torres Jun 13, 2018
4bbdeae
parallel
jose-torres Jun 13, 2018
e57531d
fix compile
jose-torres Jun 13, 2018
cff37c4
fix compile
jose-torres Jun 13, 2018
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
Next Next commit
continuous shuffle read RDD
  • Loading branch information
jose-torres committed May 15, 2018
commit 1d6b71898e2a640e3c0809695d2b83f3f84eaa38
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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 java.util.UUID

import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.util.NextIterator

case class ContinuousShuffleReadPartition(index: Int) extends Partition {
// Initialized only on the executor, and only once even as we call compute() multiple times.
lazy val (receiver, endpoint) = {
val env = SparkEnv.get.rpcEnv
val receiver = new UnsafeRowReceiver(env)
val endpoint = env.setupEndpoint(UUID.randomUUID().toString, receiver)
TaskContext.get().addTaskCompletionListener { ctx =>
env.stop(endpoint)
}
(receiver, endpoint)
}
}

/**
* RDD at the bottom of each continuous processing shuffle task, reading from the
*/
class ContinuousShuffleReadRDD(sc: SparkContext, numPartitions: Int)
extends RDD[UnsafeRow](sc, Nil) {

override protected def getPartitions: Array[Partition] = {
(0 until numPartitions).map(ContinuousShuffleReadPartition).toArray
}

override def compute(split: Partition, context: TaskContext): Iterator[UnsafeRow] = {
val receiver = split.asInstanceOf[ContinuousShuffleReadPartition].receiver

new NextIterator[UnsafeRow] {
override def getNext(): UnsafeRow = receiver.poll() match {
case ReceiverRow(r) => r
case ReceiverEpochMarker() =>
finished = true
null
}

override def close(): Unit = {}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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 java.util.concurrent.{ArrayBlockingQueue, BlockingQueue}
import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.internal.Logging
import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.sql.catalyst.expressions.UnsafeRow

/**
* Messages for the UnsafeRowReceiver endpoint. Either an incoming row or an epoch marker.
*/
private[shuffle] sealed trait UnsafeRowReceiverMessage extends Serializable
private[shuffle] case class ReceiverRow(row: UnsafeRow) extends UnsafeRowReceiverMessage
private[shuffle] case class ReceiverEpochMarker() extends UnsafeRowReceiverMessage

/**
* RPC endpoint for receiving rows into a continuous processing shuffle task.
*/
private[shuffle] class UnsafeRowReceiver(val rpcEnv: RpcEnv)
extends ThreadSafeRpcEndpoint with Logging {
private val queue = new ArrayBlockingQueue[UnsafeRowReceiverMessage](1024)
var stopped = new AtomicBoolean(false)

override def onStop(): Unit = {
stopped.set(true)
}

override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
case r: UnsafeRowReceiverMessage =>
queue.put(r)
context.reply(())
}

/**
* Polls until a new row is available.
*/
def poll(): UnsafeRowReceiverMessage = queue.poll()
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/*
* 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.{TaskContext, TaskContextImpl}
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection}
import org.apache.spark.sql.streaming.StreamTest
import org.apache.spark.sql.types.{DataType, IntegerType}

class ContinuousShuffleReadSuite extends StreamTest {

private def unsafeRow(value: Int) = {
UnsafeProjection.create(Array(IntegerType : DataType))(
new GenericInternalRow(Array(value: Any)))
}

var ctx: TaskContextImpl = _

override def beforeEach(): Unit = {
super.beforeEach()
ctx = TaskContext.empty()
TaskContext.setTaskContext(ctx)
}

override def afterEach(): Unit = {
ctx.markTaskCompleted(None)
ctx = null
super.afterEach()
}

test("receiver stopped with row last") {
val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1)
val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
endpoint.askSync[Unit](ReceiverEpochMarker())
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))

ctx.markTaskCompleted(None)
val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].receiver
eventually(timeout(streamingTimeout)) {
assert(receiver.stopped.get())
}
}

test("receiver stopped with marker last") {
val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1)
val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))
endpoint.askSync[Unit](ReceiverEpochMarker())

ctx.markTaskCompleted(None)
val receiver = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].receiver
eventually(timeout(streamingTimeout)) {
assert(receiver.stopped.get())
}
}

test("one epoch") {
val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1)
val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))
endpoint.askSync[Unit](ReceiverRow(unsafeRow(222)))
endpoint.askSync[Unit](ReceiverRow(unsafeRow(333)))
endpoint.askSync[Unit](ReceiverEpochMarker())

val iter = rdd.compute(rdd.partitions(0), ctx)
assert(iter.next().getInt(0) == 111)
assert(iter.next().getInt(0) == 222)
assert(iter.next().getInt(0) == 333)
assert(!iter.hasNext)
}

test("multiple epochs") {
val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1)
val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))
endpoint.askSync[Unit](ReceiverEpochMarker())
endpoint.askSync[Unit](ReceiverRow(unsafeRow(222)))
endpoint.askSync[Unit](ReceiverRow(unsafeRow(333)))
endpoint.askSync[Unit](ReceiverEpochMarker())

val firstEpoch = rdd.compute(rdd.partitions(0), ctx)
assert(firstEpoch.next().getInt(0) == 111)
assert(!firstEpoch.hasNext)

val secondEpoch = rdd.compute(rdd.partitions(0), ctx)
assert(secondEpoch.next().getInt(0) == 222)
assert(secondEpoch.next().getInt(0) == 333)
assert(!secondEpoch.hasNext)
}

test("empty epochs") {
val rdd = new ContinuousShuffleReadRDD(sparkContext, numPartitions = 1)
val endpoint = rdd.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
endpoint.askSync[Unit](ReceiverEpochMarker())
endpoint.askSync[Unit](ReceiverEpochMarker())
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))
endpoint.askSync[Unit](ReceiverEpochMarker())
endpoint.askSync[Unit](ReceiverEpochMarker())

assert(rdd.compute(rdd.partitions(0), ctx).isEmpty)
assert(rdd.compute(rdd.partitions(0), ctx).isEmpty)
val thirdEpoch = rdd.compute(rdd.partitions(0), ctx)
assert(thirdEpoch.next().getInt(0) == 111)
assert(rdd.compute(rdd.partitions(0), ctx).isEmpty)
assert(rdd.compute(rdd.partitions(0), ctx).isEmpty)
}
}