Skip to content

Conversation

@jose-torres
Copy link
Contributor

What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR #20552, we've changed and strictly defined the lifecycle of writer components. This means we no longer need the complicated lifecycle shim from that PR; it just naturally works.

How was this patch tested?

existing tests

@SparkQA
Copy link

SparkQA commented Mar 31, 2018

Test build #88764 has finished for PR 20951 at commit f792d32.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

* @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

val input = MemoryStream[Int]
val query = input.toDS().repartition(1).writeStream
.option("checkpointLocation", checkpointDir.getCanonicalPath)
.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

}
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)

@tdas
Copy link
Contributor

tdas commented Apr 2, 2018

LGTM. Just one comment.

@SparkQA
Copy link

SparkQA commented Apr 2, 2018

Test build #88833 has finished for PR 20951 at commit f2c3408.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter

@SparkQA
Copy link

SparkQA commented Apr 3, 2018

Test build #88832 has finished for PR 20951 at commit ebd2580.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@asfgit asfgit closed this in 66a3a5a Apr 3, 2018
robert3005 pushed a commit to palantir/spark that referenced this pull request Apr 4, 2018
## What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR apache#20552, we've changed and strictly defined the lifecycle of writer components. This means we no longer need the complicated lifecycle shim from that PR; it just naturally works.

## How was this patch tested?

existing tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes apache#20951 from jose-torres/foreach.
mshtelma pushed a commit to mshtelma/spark that referenced this pull request Apr 5, 2018
## What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR apache#20552, we've changed and strictly defined the lifecycle of writer components. This means we no longer need the complicated lifecycle shim from that PR; it just naturally works.

## How was this patch tested?

existing tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes apache#20951 from jose-torres/foreach.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants