-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23099][SS] Migrate foreach sink to DataSourceV2 #20951
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 5 commits
a617186
6ddac36
f792d32
117a30a
ebd2580
d62ffa1
a2e9eb2
f2c3408
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,111 @@ | ||
| /* | ||
| * 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.sources | ||
|
|
||
| import org.apache.spark.sql.{Encoder, ForeachWriter, SparkSession} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} | ||
| 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 [[org.apache.spark.sql.sources.v2.DataSourceV2]] for forwarding data into the specified | ||
| * [[ForeachWriter]]. | ||
| * | ||
| * @param writer The [[ForeachWriter]] to process all data. | ||
| * @tparam T The expected type of the sink. | ||
| */ | ||
| case class ForeachWriterProvider[T: Encoder](writer: ForeachWriter[T]) extends StreamWriteSupport { | ||
| 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 createInternalRowWriterFactory(): DataWriterFactory[InternalRow] = { | ||
| val encoder = encoderFor[T].resolveAndBind( | ||
| schema.toAttributes, | ||
| SparkSession.getActiveSession.get.sessionState.analyzer) | ||
| ForeachWriterFactory(writer, encoder) | ||
| } | ||
|
|
||
| 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) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * A [[DataWriter]] which writes data in this partition to a [[ForeachWriter]]. | ||
| * @param writer The [[ForeachWriter]] to process all data. | ||
| * @param encoder An encoder which can convert [[InternalRow]] to the required type [[T]] | ||
| * @param partitionId | ||
| * @param epochId | ||
| * @tparam T The type expected by the writer. | ||
| */ | ||
| 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 |
|---|---|---|
|
|
@@ -131,7 +131,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf | |
| .foreach(new TestForeachWriter() { | ||
| 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) | ||
|
|
@@ -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") | ||
|
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. why 3 levels? Can you paste the levels here in the PR comments?
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. [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] Cause: org.apache.spark.SparkException: Writing job aborted. [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] Cause: java.lang.RuntimeException: ForeachSinkSuite error |
||
| assert(query.isActive === false) | ||
|
|
||
| val allEvents = ForeachSinkSuite.allEvents() | ||
|
|
@@ -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") | ||
| } | ||
| } | ||
|
|
||
|
|
||
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.
Maybe rename this to
ForeachWriterSuite?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.
And move this to streaming.sources package similar ConsoleWriterSuite