-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-38323][SQL][Streaming] Support the hidden file metadata in Streaming #35676
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
Conversation
|
@cloud-fan appreciate it if you can take a look, thanks! |
|
Can you fill the |
|
Can one of the admins verify this patch? |
|
thanks, merging to master! |
| ) | ||
|
|
||
| // SELECT * will have: name, age, info, _metadata of /source/new-streaming-data | ||
| assert(newDF.select("*").columns.toSet == Set("name", "age", "info", "_metadata")) |
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.
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
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.
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.
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.
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.
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.
I see! I can test with dropDuplicates() and see how it goes
|
After manual testing locally, changes of
|
|
Thanks for checking it manually! We are good to go. |
### 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>
### 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>
### 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>
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:
Why are the changes needed?
Add more support to the hidden file metadata feature. Before this PR, querying the hidden file metadata struct
_metadatawill fail usingreadStream,writeStreamstreaming APIs.Does this PR introduce any user-facing change?
No
How was this patch tested?
Add a new UT