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
Prev Previous commit
Next Next commit
add interface
  • Loading branch information
jose-torres committed May 18, 2018
commit 97f7e8ff865e6054d0d70914ce9bb51880b161f6
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator

case class ContinuousShuffleReadPartition(index: Int, queueSize: Int) extends Partition {
// Initialized only on the executor, and only once even as we call compute() multiple times.
lazy val (receiver, endpoint) = {
lazy val (reader: ContinuousShuffleReader, endpoint) = {
val env = SparkEnv.get.rpcEnv
val receiver = new UnsafeRowReceiver(queueSize, env)
val endpoint = env.setupEndpoint(s"UnsafeRowReceiver-${UUID.randomUUID().toString}", receiver)
Expand All @@ -53,17 +53,6 @@ class ContinuousShuffleReadRDD(sc: SparkContext, numPartitions: Int)
}

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

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

override def close(): Unit = {}
}
split.asInstanceOf[ContinuousShuffleReadPartition].reader.read()
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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 reading from a continuous processing shuffle.
*/
trait ContinuousShuffleReader {
/**
* Returns an iterator over the incoming rows in the current epoch. Note that this iterator can
* block waiting for new rows to arrive.
*/
def read(): Iterator[UnsafeRow]
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ 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
import org.apache.spark.util.NextIterator

/**
* Messages for the UnsafeRowReceiver endpoint. Either an incoming row or an epoch marker.
Expand All @@ -41,7 +42,7 @@ private[shuffle] case class ReceiverEpochMarker() extends UnsafeRowReceiverMessa
private[shuffle] class UnsafeRowReceiver(
queueSize: Int,
override val rpcEnv: RpcEnv)
extends ThreadSafeRpcEndpoint with Logging {
extends ThreadSafeRpcEndpoint with ContinuousShuffleReader with Logging {
// Note that this queue will be drained from the main task thread and populated in the RPC
// response thread.
private val queue = new ArrayBlockingQueue[UnsafeRowReceiverMessage](queueSize)
Expand All @@ -59,8 +60,16 @@ private[shuffle] class UnsafeRowReceiver(
context.reply(())
}

/**
* Take the next row, blocking until it's ready.
*/
def take(): UnsafeRowReceiverMessage = queue.take()
override def read(): Iterator[UnsafeRow] = {
new NextIterator[UnsafeRow] {
override def getNext(): UnsafeRow = queue.take() match {
case ReceiverRow(r) => r
case ReceiverEpochMarker() =>
finished = true
null
}

override def close(): Unit = {}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,9 +54,9 @@ class ContinuousShuffleReadSuite extends StreamTest {
endpoint.askSync[Unit](ReceiverRow(unsafeRow(111)))

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

Expand All @@ -67,9 +67,9 @@ class ContinuousShuffleReadSuite extends StreamTest {
endpoint.askSync[Unit](ReceiverEpochMarker())

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

Expand Down