Skip to content

Conversation

@xuanyuanking
Copy link
Member

What changes were proposed in this pull request?

This pr aims to add a new table API in DataStreamReader, which is similar to the table API in DataFrameReader.

Why are the changes needed?

Users can directly use this API to get a Streaming DataFrame on a table. Below is a simple example:

Application 1 for initializing and starting the streaming job:

val path = "/home/yuanjian.li/runtime/to_be_deleted"
val tblName = "my_table"

// Write some data to `my_table`
spark.range(3).write.format("parquet").option("path", path).saveAsTable(tblName)

// Read the table as a streaming source, write result to destination directory
val table = spark.readStream.table(tblName)
table.writeStream.format("parquet").option("checkpointLocation", "/home/yuanjian.li/runtime/to_be_deleted_ck").start("/home/yuanjian.li/runtime/to_be_deleted_2")

Application 2 for appending new data:

// Append new data into the path
spark.range(5).write.format("parquet").option("path", "/home/yuanjian.li/runtime/to_be_deleted").mode("append").save()

Check result:

// The desitination directory should contains all written data
spark.read.parquet("/home/yuanjian.li/runtime/to_be_deleted_2").show()

Does this PR introduce any user-facing change?

Yes, a new API added.

How was this patch tested?

New UT added and integrated testing.

case NonSessionCatalogAndIdentifier(catalog, ident) =>
CatalogV2Util.loadTable(catalog, ident) match {
case Some(table) =>
Some(StreamingRelationV2(
Copy link
Member Author

Choose a reason for hiding this comment

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

With the refactory of StreamingRelationV2(#29633), we can directly create it in the catalyst.

Copy link
Contributor

Choose a reason for hiding this comment

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

shall we just add a isStreaming flag in lookupV2Relation, to unify the code a bit more?

Copy link
Member Author

Choose a reason for hiding this comment

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

Sure, done in 305c316

private def getStreamingRelation(
table: CatalogTable,
extraOptions: CaseInsensitiveStringMap): StreamingRelation = {
val dsOptions = DataSourceUtils.generateDatasourceOptions(extraOptions, table)
Copy link
Member Author

Choose a reason for hiding this comment

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

Keep the same behavior with DataFrameReader.table on respecting options(#29712)

@SparkQA
Copy link

SparkQA commented Sep 15, 2020

Test build #128699 has finished for PR 29756 at commit 43a371d.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 15, 2020

Test build #128701 has finished for PR 29756 at commit 97c3e0b.

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

@xuanyuanking
Copy link
Member Author

@HeartSaVioR
Copy link
Contributor

Thanks for the patch. I was also about to find the missing spot (as I commented in #29715) and this PR looks to fulfill the needs. I'll take a look soon.

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

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

I'll review once there're enough tests to cover the functionality, as that would save plenty of times on reviewing.

@HeartSaVioR
Copy link
Contributor

Btw to expand the tests you may also want to have the changes on InMemoryTable. Would it be good for you to go through my PR first and rebase, or let me extract the part to the separate PR?

child.collect {
// Disallow creating permanent views based on temporary views.
case UnresolvedRelation(nameParts, _) if catalog.isTempView(nameParts) =>
case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) =>
Copy link
Member

Choose a reason for hiding this comment

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

isStreaming = false only?

Copy link
Member Author

Choose a reason for hiding this comment

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

We can create a temp view based on streaming relation, so it should be kept as a full match?

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

Comment on lines +480 to +482
* Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should
* support streaming mode.
Copy link
Member

Choose a reason for hiding this comment

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

If the data source doesn't support, what will happen?

Copy link
Member Author

Choose a reason for hiding this comment

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

It will check in MicroBatchExecution/ContinuousExecution if the data source doesn't support.

}
}

class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
Copy link
Member

@zsxwing zsxwing Sep 17, 2020

Choose a reason for hiding this comment

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

nit: could you move this to a new file when you add more tests?

Copy link
Member Author

Choose a reason for hiding this comment

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

Sure, adding more tests now.

@xuanyuanking
Copy link
Member Author

Would it be good for you to go through my PR first and rebase, or let me extract the part to the separate PR?

@HeartSaVioR Sure, let's go through the writer's side first. I think it's ok to rebase or resolve conflicts.

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
case u @ UnresolvedRelation(ident, _) =>
case u @ UnresolvedRelation(ident, _, _) =>
lookupTempView(ident).getOrElse(u)
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we fail if the temp view is not a streaming plan but the isStreaming flag is true?

Copy link
Contributor

Choose a reason for hiding this comment

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

Probably vice versa.

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure about the other way around. We have SparkSession.table, which can read both batch or streaming temp views. We shouldn't break it.

It's a bit weird if DataFramaReader.table can read streaming temp view, but this is the existing behavior and probably is fine.

Copy link
Contributor

@HeartSaVioR HeartSaVioR Sep 17, 2020

Choose a reason for hiding this comment

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

Just curious, is it end user's responsibility to know about whether the temp view is from batch or streaming, so that they can correctly call write or writeStream? Without thinking of SparkSession.table I assume it's clear as end user will match the reader side and writer side clearly (read/write or readStream/writeStream), and it looks a bit confusing.

If DataFrameReader.table allows streaming temp view, then I guess read/writeStream pair is possible which is a bit confusing. (or does it change the plan to the batch one magically?)

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it's less confusing if DataFrameReader.table fails on reading streaming temp view while SparkSession.table works. But it's a breaking change and we at least shouldn't do it in this PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

OK I agree it should be better to fix in different PR. I still think we shouldn't support confused things just because we supported them. The fix would depend on change of the PR (isStreaming flag) - I'll wait for this PR and try to fix it after the PR.

)
}

test("stream table API support") {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we move DataStreamTableAPISuite to a new file and move this test to there as well?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, done in 9004fba

@SparkQA
Copy link

SparkQA commented Sep 17, 2020

Test build #128820 has finished for PR 29756 at commit 305c316.

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

@SparkQA
Copy link

SparkQA commented Sep 21, 2020

Test build #128931 has finished for PR 29756 at commit 9004fba.

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

@SparkQA
Copy link

SparkQA commented Sep 23, 2020

Test build #129019 has finished for PR 29756 at commit 03a91f8.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds the following public classes (experimental):
  • trait V2TableWithV1Fallback extends Table

@SparkQA
Copy link

SparkQA commented Sep 23, 2020

Test build #129020 has finished for PR 29756 at commit fad1976.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • trait V2TableWithV1Fallback extends Table

Copy link
Contributor

@cloud-fan cloud-fan left a comment

Choose a reason for hiding this comment

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

LGTM except some minor comments

@SparkQA
Copy link

SparkQA commented Sep 24, 2020

Test build #129062 has finished for PR 29756 at commit 97761d2.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • trait V2TableWithV1Fallback extends Table

@SparkQA
Copy link

SparkQA commented Sep 24, 2020

Test build #129069 has finished for PR 29756 at commit d2eb23f.

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

@cloud-fan
Copy link
Contributor

thanks, merging to master!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants