Skip to content

Conversation

@Yaohua628
Copy link
Contributor

@Yaohua628 Yaohua628 commented Feb 28, 2022

What changes were proposed in this pull request?

Support the hidden file source metadata in streaming, you could do the same thing as batch read/write as follows:

spark
  .readStream
  ...
  .select("*", "_metadata")
  .writeStream
  ...
  .start()

Why are the changes needed?

Add more support to the hidden file metadata feature. Before this PR, querying the hidden file metadata struct _metadata will fail using readStream, writeStream streaming APIs.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Add a new UT

@Yaohua628
Copy link
Contributor Author

@cloud-fan appreciate it if you can take a look, thanks!

@cloud-fan
Copy link
Contributor

Can you fill the What changes were proposed in this pull request? section and fix merge conflicts? The code change looks good. Also cc @HeartSaVioR

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in ad4e5a6 Mar 1, 2022
)

// SELECT * will have: name, age, info, _metadata of /source/new-streaming-data
assert(newDF.select("*").columns.toSet == Set("name", "age", "info", "_metadata"))
Copy link
Contributor

@HeartSaVioR HeartSaVioR Mar 3, 2022

Choose a reason for hiding this comment

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

@Yaohua628

Sorry for the post review. I haven't had a time to review this in time.

Just to make clear, select("*").show() should not expose a hidden column, right? Since you've included "_metadata" from the list of columns so I would like to double confirm that it is not user facing.

And given we include the new column, dropDuplicate without explicitly mentioning columns in streaming query would be broken. state schema would somehow include the hidden column in the schema, whereas state schema from older version of checkpoint does not include the hidden column (as they didn't exist).

We should test it, and if it fall into the case, we should mention this in the migration guide, or make this configurable and by default turn off. (We did this for adding Kafka header - #22282)

cc. @cloud-fan

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yep, select("*") won't expose the hidden file metadata column.

But here what I did is readStream and explicitly selecting * and _metadata (here) and writeStream to a target table /target/new-streaming-data, so the target table itself will have a column called _metadata.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah OK my bad. That is just checking the output.

dropDuplicate() still remains a question. How we deal with this? We removed the column by default for Kafka header to not break compatibility with dropDuplicate(), but not sure we would like to add more config.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I see! I can test with dropDuplicates() and see how it goes

@Yaohua628
Copy link
Contributor Author

After manual testing locally, changes of _metadata in streaming won't cause any backward-compatible issues for dropDuplicates():

  1. with Spark 3.2: readStream...select("*").dropDuplicates().writeStream...option("checkpointLocation", ".../checkpoint")
  2. verify results (duplicated data did get dropped), verify checkpoint dir (commits, offsets, sources, state)
  3. add some new files including duplicates
  4. with Spark master (3.3 with _metadata feature) with the same checkpoint location: readStream...select("*").dropDuplicates().writeStream...option("checkpointLocation", ".../checkpoint")
  5. verify results (duplicated data did get dropped), verify checkpoint dir (commits, offsets, sources, state)

@HeartSaVioR
Copy link
Contributor

Thanks for checking it manually! We are good to go.

HeartSaVioR pushed a commit that referenced this pull request Jun 8, 2022
### What changes were proposed in this pull request?
We added the support to query the `_metadata` column with a file-based streaming source: #35676.

We propose to use `transformUp` instead of `match` when pattern matching the `dataPlan` in `MicroBatchExecution` `runBatch` method in this PR. It is fine for `FileStreamSource` because `FileStreamSource` always returns one `LogicalRelation` node (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L247).

But the proposed change will make the logic robust and we really should not rely on the upstream source to return a desired plan. In addition, the proposed change could also make `_metadata` work if someone wants to customize `FileStreamSource` `getBatch`.

### Why are the changes needed?
Robust

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests

Closes #36801 from Yaohua628/spark-39404.

Authored-by: yaohua <yaohua.zhao@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Yaohua628 added a commit to Yaohua628/spark that referenced this pull request Oct 22, 2022
### What changes were proposed in this pull request?
We added the support to query the `_metadata` column with a file-based streaming source: apache#35676.

We propose to use `transformUp` instead of `match` when pattern matching the `dataPlan` in `MicroBatchExecution` `runBatch` method in this PR. It is fine for `FileStreamSource` because `FileStreamSource` always returns one `LogicalRelation` node (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L247).

But the proposed change will make the logic robust and we really should not rely on the upstream source to return a desired plan. In addition, the proposed change could also make `_metadata` work if someone wants to customize `FileStreamSource` `getBatch`.

### Why are the changes needed?
Robust

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests

Closes apache#36801 from Yaohua628/spark-39404.

Authored-by: yaohua <yaohua.zhao@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
HeartSaVioR pushed a commit that referenced this pull request Oct 22, 2022
### What changes were proposed in this pull request?

(This cherry-picks #36801)

We added the support to query the `_metadata` column with a file-based streaming source: #35676.

We propose to use `transformUp` instead of `match` when pattern matching the `dataPlan` in `MicroBatchExecution` `runBatch` method in this PR. It is fine for `FileStreamSource` because `FileStreamSource` always returns one `LogicalRelation` node (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L247).

But the proposed change will make the logic robust and we really should not rely on the upstream source to return a desired plan. In addition, the proposed change could also make `_metadata` work if someone wants to customize `FileStreamSource` `getBatch`.

### Why are the changes needed?
Robust

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests

Closes #38337 from Yaohua628/spark-39404-3-3.

Authored-by: yaohua <yaohua.zhao@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
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.

4 participants