Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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 @@ -17,52 +17,81 @@

package org.apache.spark.sql.execution.streaming

import org.apache.spark.TaskContext
import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter}
import org.apache.spark.sql.{Encoder, ForeachWriter, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.encoders.encoderFor
import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamWriteSupport}
import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory, SupportsWriteInternalRow, WriterCommitMessage}
import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType

/**
* A [[Sink]] that forwards all data into [[ForeachWriter]] according to the contract defined by
* [[ForeachWriter]].
*
* @param writer The [[ForeachWriter]] to process all data.
* @tparam T The expected type of the sink.
*/
class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable {
case class ForeachWriterProvider[T: Encoder](writer: ForeachWriter[T]) extends StreamWriteSupport {
Copy link
Contributor

@tdas tdas Apr 2, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rename the file accordingly. and Add docs. Clarify why this is not a DataSource but still extends StreamWriteSupport

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, why not make it extend DataSourceV2 for consistency sake? Then it is easier to find all data sources in code by looking at who extends DataSourceV2

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a DataSourceV2 - that interface extends it

override def createStreamWriter(
queryId: String,
schema: StructType,
mode: OutputMode,
options: DataSourceOptions): StreamWriter = {
new StreamWriter with SupportsWriteInternalRow {
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}

override def addBatch(batchId: Long, data: DataFrame): Unit = {
// This logic should've been as simple as:
// ```
// data.as[T].foreachPartition { iter => ... }
// ```
//
// Unfortunately, doing that would just break the incremental planing. The reason is,
// `Dataset.foreachPartition()` would further call `Dataset.rdd()`, but `Dataset.rdd()` will
// create a new plan. Because StreamExecution uses the existing plan to collect metrics and
// update watermark, we should never create a new plan. Otherwise, metrics and watermark are
// updated in the new plan, and StreamExecution cannot retrieval them.
//
// Hence, we need to manually convert internal rows to objects using encoder.
val encoder = encoderFor[T].resolveAndBind(
data.logicalPlan.output,
data.sparkSession.sessionState.analyzer)
data.queryExecution.toRdd.foreachPartition { iter =>
if (writer.open(TaskContext.getPartitionId(), batchId)) {
try {
while (iter.hasNext) {
writer.process(encoder.fromRow(iter.next()))
}
} catch {
case e: Throwable =>
writer.close(e)
throw e
}
writer.close(null)
} else {
writer.close(null)
override def createInternalRowWriterFactory(): DataWriterFactory[InternalRow] = {
val encoder = encoderFor[T].resolveAndBind(
schema.toAttributes,
SparkSession.getActiveSession.get.sessionState.analyzer)
ForeachWriterFactory(writer, encoder)
}

override def toString: String = "ForeachSink"
}
}
}

override def toString(): String = "ForeachSink"
case class ForeachWriterFactory[T: Encoder](
writer: ForeachWriter[T],
encoder: ExpressionEncoder[T])
extends DataWriterFactory[InternalRow] {
override def createDataWriter(
partitionId: Int,
attemptNumber: Int,
epochId: Long): ForeachDataWriter[T] = {
new ForeachDataWriter(writer, encoder, partitionId, epochId)
}
}

class ForeachDataWriter[T : Encoder](
writer: ForeachWriter[T],
encoder: ExpressionEncoder[T],
partitionId: Int,
epochId: Long)
extends DataWriter[InternalRow] {

// If open returns false, we should skip writing rows.
private val opened = writer.open(partitionId, epochId)

override def write(record: InternalRow): Unit = {
if (!opened) return

try {
writer.process(encoder.fromRow(record))
} catch {
case t: Throwable =>
writer.close(t)
throw t
}
}

override def commit(): WriterCommitMessage = {
writer.close(null)
ForeachWriterCommitMessage
}

override def abort(): Unit = {}
}

/**
* An empty [[WriterCommitMessage]]. [[ForeachWriter]] implementations have no global coordination.
*/
case object ForeachWriterCommitMessage extends WriterCommitMessage
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
query
} else if (source == "foreach") {
assertNotPartitioned("foreach")
val sink = new ForeachSink[T](foreachWriter)(ds.exprEnc)
val sink = new ForeachWriterProvider[T](foreachWriter)(ds.exprEnc)
df.sparkSession.sessionState.streamingQueryManager.startQuery(
extraOptions.get("queryName"),
extraOptions.get("checkpointLocation"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf
.foreach(new TestForeachWriter() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe rename this to ForeachWriterSuite?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And move this to streaming.sources package similar ConsoleWriterSuite

override def process(value: Int): Unit = {
super.process(value)
throw new RuntimeException("error")
throw new RuntimeException("ForeachSinkSuite error")
}
}).start()
input.addData(1, 2, 3, 4)
Expand All @@ -141,7 +141,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf
query.processAllAvailable()
}
assert(e.getCause.isInstanceOf[SparkException])
assert(e.getCause.getCause.getMessage === "error")
assert(e.getCause.getCause.getCause.getMessage === "ForeachSinkSuite error")
Copy link
Contributor

@tdas tdas Apr 2, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why 3 levels? Can you paste the levels here in the PR comments?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[info] org.apache.spark.sql.streaming.StreamingQueryException: Query [id = c80c8860-d4f5-47c6-9a2b-33b5172e1735, runId = 81acd408-9028-41ee-9349-866ae2d67615] terminated with exception: Writing job aborted.
[info] at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)

[info] Cause: org.apache.spark.SparkException: Writing job aborted.
[info] at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2.scala:117)

[info] Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, localhost, executor driver): java.lang.RuntimeException: ForeachSinkSuite error
[info] at org.apache.spark.sql.execution.streaming.sources.ForeachWriterSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anon$1.process(ForeachWriterSuite.scala:135)

[info] Cause: java.lang.RuntimeException: ForeachSinkSuite error
[info] at org.apache.spark.sql.execution.streaming.sources.ForeachWriterSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anon$1.process(ForeachWriterSuite.scala:135)

assert(query.isActive === false)

val allEvents = ForeachSinkSuite.allEvents()
Expand All @@ -152,7 +152,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf
// `close` should be called with the error
val errorEvent = allEvents(0)(2).asInstanceOf[ForeachSinkSuite.Close]
assert(errorEvent.error.get.isInstanceOf[RuntimeException])
assert(errorEvent.error.get.getMessage === "error")
assert(errorEvent.error.get.getMessage === "ForeachSinkSuite error")
}
}

Expand Down