Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -346,6 +346,7 @@ object UnsupportedOperationChecker {
subPlan match {
case (_: Project | _: Filter | _: MapElements | _: MapPartitions |
_: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias) =>
case _: Aggregate => // TODO
case node if node.nodeName == "StreamingRelationV2" =>
case node =>
throwError(s"Continuous processing does not support ${node.nodeName} operations.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ class ContinuousDataSourceRDD(
@transient private val readerFactories: Seq[DataReaderFactory[UnsafeRow]])
extends RDD[UnsafeRow](sc, Nil) {

private var readerForTask: ContinuousReaderForTask = _
private val dataQueueSize = sqlContext.conf.continuousStreamingExecutorQueueSize
private val epochPollIntervalMs = sqlContext.conf.continuousStreamingExecutorPollIntervalMs

Expand All @@ -53,72 +54,47 @@ class ContinuousDataSourceRDD(
throw new ContinuousTaskRetryException()
}

val reader = split.asInstanceOf[DataSourceRDDPartition[UnsafeRow]]
.readerFactory.createDataReader()
if (readerForTask == null) {
readerForTask =
new ContinuousReaderForTask(split, context, dataQueueSize, epochPollIntervalMs)
}

val coordinatorId = context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY)

// This queue contains two types of messages:
// * (null, null) representing an epoch boundary.
// * (row, off) containing a data row and its corresponding PartitionOffset.
val queue = new ArrayBlockingQueue[(UnsafeRow, PartitionOffset)](dataQueueSize)

val epochPollFailed = new AtomicBoolean(false)
val epochPollExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
s"epoch-poll--$coordinatorId--${context.partitionId()}")
val epochPollRunnable = new EpochPollRunnable(queue, context, epochPollFailed)
epochPollExecutor.scheduleWithFixedDelay(
epochPollRunnable, 0, epochPollIntervalMs, TimeUnit.MILLISECONDS)

// Important sequencing - we must get start offset before the data reader thread begins
val startOffset = ContinuousDataSourceRDD.getBaseReader(reader).getOffset

val dataReaderFailed = new AtomicBoolean(false)
val dataReaderThread = new DataReaderThread(reader, queue, context, dataReaderFailed)
dataReaderThread.setDaemon(true)
dataReaderThread.start()

context.addTaskCompletionListener(_ => {
dataReaderThread.interrupt()
epochPollExecutor.shutdown()
})

val epochEndpoint = EpochCoordinatorRef.get(coordinatorId, SparkEnv.get)
new Iterator[UnsafeRow] {
private val POLL_TIMEOUT_MS = 1000

private var currentEntry: (UnsafeRow, PartitionOffset) = _
private var currentOffset: PartitionOffset = startOffset
private var currentEpoch =
context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong

override def hasNext(): Boolean = {
while (currentEntry == null) {
if (context.isInterrupted() || context.isCompleted()) {
currentEntry = (null, null)
}
if (dataReaderFailed.get()) {
throw new SparkException("data read failed", dataReaderThread.failureReason)
if (readerForTask.dataReaderFailed.get()) {
throw new SparkException(
"data read failed", readerForTask.dataReaderThread.failureReason)
}
if (epochPollFailed.get()) {
throw new SparkException("epoch poll failed", epochPollRunnable.failureReason)
if (readerForTask.epochPollFailed.get()) {
throw new SparkException(
"epoch poll failed", readerForTask.epochPollRunnable.failureReason)
}
currentEntry = queue.poll(POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS)
currentEntry = readerForTask.queue.poll(POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS)
}

currentEntry match {
// epoch boundary marker
case (null, null) =>
epochEndpoint.send(ReportPartitionOffset(
context.partitionId(),
currentEpoch,
currentOffset))
currentEpoch += 1
readerForTask.currentEpoch,
readerForTask.currentOffset))
readerForTask.currentEpoch += 1
currentEntry = null
false
// real row
case (_, offset) =>
currentOffset = offset
readerForTask.currentOffset = offset
true
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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

import java.util.concurrent.{ArrayBlockingQueue, TimeUnit}
import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.{Partition, TaskContext}
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition
import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset
import org.apache.spark.util.ThreadUtils

/**
* An object containing a queue of continuous processing records read in.
* Instantiated once per task.
*/
class ContinuousReaderForTask(
split: Partition,
context: TaskContext,
dataQueueSize: Int,
epochPollIntervalMs: Long) {
// This queue contains two types of messages:
// * (null, null) representing an epoch boundary.
// * (row, off) containing a data row and its corresponding PartitionOffset.
val queue = new ArrayBlockingQueue[(UnsafeRow, PartitionOffset)](dataQueueSize)

val epochPollFailed = new AtomicBoolean(false)
val dataReaderFailed = new AtomicBoolean(false)

private val reader = split.asInstanceOf[DataSourceRDDPartition[UnsafeRow]]
.readerFactory.createDataReader()

private val coordinatorId = context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY)

private val epochPollExecutor = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
s"epoch-poll--$coordinatorId--${context.partitionId()}")
val epochPollRunnable = new EpochPollRunnable(queue, context, epochPollFailed)
epochPollExecutor.scheduleWithFixedDelay(
epochPollRunnable, 0, epochPollIntervalMs, TimeUnit.MILLISECONDS)

// Important sequencing - we must get start offset before the data reader thread begins
var currentOffset = ContinuousDataSourceRDD.getBaseReader(reader).getOffset
var currentEpoch =
context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong

val dataReaderThread = new DataReaderThread(reader, queue, context, dataReaderFailed)
dataReaderThread.setDaemon(true)
dataReaderThread.start()

context.addTaskCompletionListener(_ => {
dataReaderThread.interrupt()
epochPollExecutor.shutdown()
})
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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

import org.apache.spark.{Partition, SparkEnv, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask.{logError, logInfo}
import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory, WriterCommitMessage}
import org.apache.spark.util.Utils

class ContinuousWriteRDD(var prev: RDD[InternalRow], writeTask: DataWriterFactory[InternalRow])
extends RDD[Unit](prev) {

override val partitioner = prev.partitioner

override def getPartitions: Array[Partition] = prev.partitions

override def compute(split: Partition, context: TaskContext): Iterator[Unit] = {
val epochCoordinator = EpochCoordinatorRef.get(
context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
SparkEnv.get)
var currentEpoch = context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong

do {
var dataWriter: DataWriter[InternalRow] = null
// write the data and commit this writer.
Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
try {
val dataIterator = prev.compute(split, context)
dataWriter = writeTask.createDataWriter(
context.partitionId(), context.attemptNumber(), currentEpoch)
while (dataIterator.hasNext) {
dataWriter.write(dataIterator.next())
}
logInfo(s"Writer for partition ${context.partitionId()} " +
s"in epoch $currentEpoch is committing.")
val msg = dataWriter.commit()
epochCoordinator.send(
CommitPartitionEpoch(context.partitionId(), currentEpoch, msg)
)
logInfo(s"Writer for partition ${context.partitionId()} " +
s"in epoch $currentEpoch committed.")
currentEpoch += 1
} catch {
case _: InterruptedException =>
// Continuous shutdown always involves an interrupt. Just finish the task.
}
})(catchBlock = {
// If there is an error, abort this writer. We enter this callback in the middle of
// rethrowing an exception, so compute() will stop executing at this point.
logError(s"Writer for partition ${context.partitionId()} is aborting.")
if (dataWriter != null) dataWriter.abort()
logError(s"Writer for partition ${context.partitionId()} aborted.")
})
} while (!context.isInterrupted() && !context.isCompleted())

Iterator()
}

override def clearDependencies() {
super.clearDependencies()
prev = null
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -46,28 +46,34 @@ case class WriteToContinuousDataSourceExec(writer: StreamWriter, query: SparkPla
case _ => new InternalRowDataWriterFactory(writer.createWriterFactory(), query.schema)
}

val rdd = query.execute()
val rdd = new ContinuousWriteRDD(query.execute(), writerFactory)
val messages = new Array[WriterCommitMessage](rdd.partitions.length)

logInfo(s"Start processing data source writer: $writer. " +
s"The input RDD has ${rdd.getNumPartitions} partitions.")
// Let the epoch coordinator know how many partitions the write RDD has.
s"The input RDD has ${messages.length} partitions.")
EpochCoordinatorRef.get(
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
sparkContext.env)
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
sparkContext.env)
.askSync[Unit](SetWriterPartitions(rdd.getNumPartitions))

try {
// Force the RDD to run so continuous processing starts; no data is actually being collected
// to the driver, as ContinuousWriteRDD outputs nothing.
sparkContext.runJob(
rdd,
(context: TaskContext, iter: Iterator[InternalRow]) =>
WriteToContinuousDataSourceExec.run(writerFactory, context, iter),
rdd.partitions.indices)
rdd.collect()
} catch {
case _: InterruptedException =>
// Interruption is how continuous queries are ended, so accept and ignore the exception.
// Interruption is how continuous queries are ended, so accept and ignore the exception.
case cause: Throwable =>
logError(s"Data source writer $writer is aborting.")
try {
writer.abort(0, messages)
} catch {
case t: Throwable =>
logError(s"Data source writer $writer failed to abort.")
cause.addSuppressed(t)
throw new SparkException("Writing job failed.", cause)
}
logError(s"Data source writer $writer aborted.")
cause match {
// Do not wrap interruption exceptions that will be handled by streaming specially.
case _ if StreamExecution.isInterruptionException(cause) => throw cause
Expand All @@ -80,45 +86,3 @@ case class WriteToContinuousDataSourceExec(writer: StreamWriter, query: SparkPla
sparkContext.emptyRDD
}
}

object WriteToContinuousDataSourceExec extends Logging {
def run(
writeTask: DataWriterFactory[InternalRow],
context: TaskContext,
iter: Iterator[InternalRow]): Unit = {
val epochCoordinator = EpochCoordinatorRef.get(
context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
SparkEnv.get)
var currentEpoch = context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong

do {
var dataWriter: DataWriter[InternalRow] = null
// write the data and commit this writer.
Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
try {
dataWriter = writeTask.createDataWriter(
context.partitionId(), context.attemptNumber(), currentEpoch)
while (iter.hasNext) {
dataWriter.write(iter.next())
}
logInfo(s"Writer for partition ${context.partitionId()} is committing.")
val msg = dataWriter.commit()
logInfo(s"Writer for partition ${context.partitionId()} committed.")
epochCoordinator.send(
CommitPartitionEpoch(context.partitionId(), currentEpoch, msg)
)
currentEpoch += 1
} catch {
case _: InterruptedException =>
// Continuous shutdown always involves an interrupt. Just finish the task.
}
})(catchBlock = {
// If there is an error, abort this writer. We enter this callback in the middle of
// rethrowing an exception, so runContinuous will stop executing at this point.
logError(s"Writer for partition ${context.partitionId()} is aborting.")
if (dataWriter != null) dataWriter.abort()
logError(s"Writer for partition ${context.partitionId()} aborted.")
})
} while (!context.isInterrupted())
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ import org.apache.spark.util.RpcUtils
class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
extends MemoryStreamBase[A](sqlContext) with ContinuousReader with ContinuousReadSupport {
private implicit val formats = Serialization.formats(NoTypeHints)
private val NUM_PARTITIONS = 2
private val NUM_PARTITIONS = 1

protected val logicalPlan =
StreamingRelationV2(this, "memory", Map(), attributes, None)(sqlContext.sparkSession)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -341,13 +341,9 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
private def updateFromDeltaFile(version: Long, map: MapType): Unit = {
val fileToRead = deltaFile(version)
var input: DataInputStream = null
val sourceStream = try {
fm.open(fileToRead)
} catch {
case f: FileNotFoundException =>
throw new IllegalStateException(
s"Error reading delta file $fileToRead of $this: $fileToRead does not exist", f)
}
// We don't guarantee that all epoch IDs actually run, so some versions might not exist.
if (!fm.exists(fileToRead)) return
val sourceStream = fm.open(fileToRead)
try {
input = decompressStream(sourceStream)
var eof = false
Expand Down
Loading