[SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for StreamingQueryProgressWrapper#39642
[SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for StreamingQueryProgressWrapper#39642LuciferYang wants to merge 19 commits intoapache:masterfrom
StreamingQueryProgressWrapper#39642Conversation
StreamingQueryProgressWrapperStreamingQueryProgressWrapper
| val durationMs: ju.Map[String, JLong], | ||
| // If `eventTime` is no longer read-only, need change | ||
| // `StreamingQueryProgressSerializer#deserialize` to return mutable Java Map | ||
| val eventTime: ju.Map[String, String], |
There was a problem hiding this comment.
There are 6 ju.Map, durationMs and observedMetrics already use mutable Java Map, the remaining 4 Java Map should be read-only at present, but for safety, we can also directly return mutable Java Map in the deserialize method
There was a problem hiding this comment.
For safety, 6 ju.Map are assigned as mutable Map after 8e89213
StreamingQueryProgressWrapperStreamingQueryProgressWrapper
|
Will refactor after #39666 merged |
|
rebased |
|
Does this one have a chance to Spark 3.4.0 ? Or wait for the next version? |
| int64 num_rows_dropped_by_watermark = 9; | ||
| int64 num_shuffle_partitions = 10; | ||
| int64 num_state_store_instances = 11; | ||
| map<string, int64> custom_metrics = 12; |
There was a problem hiding this comment.
I am concerned about the nullability of all these maps. Shall we check/test all of them?
| } | ||
|
|
||
| test("StreamingQueryProgressWrapper") { | ||
| val normalInput = { |
There was a problem hiding this comment.
Two objects are manually created due to many fields can be null
| assert(progress.batchId == resultProcess.batchId) | ||
| assert(progress.batchDuration == resultProcess.batchDuration) | ||
| if (hasNullValue) { | ||
| assert(resultProcess.durationMs.isEmpty) |
There was a problem hiding this comment.
null map input return empty map
...e/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala
Show resolved
Hide resolved
...e/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala
Show resolved
Hide resolved
There was a problem hiding this comment.
use setJMapField need to define an additional function or use an anonymous function due this is not a simple putAll scenario.
Any other better way to write this ? @gengliangwang
| ) | ||
| } | ||
|
|
||
| private def putAllObservedMetrics( |
There was a problem hiding this comment.
Change to define a class level private function
| builder: StoreTypes.StreamingQueryProgress.Builder, | ||
| observedMetrics: JMap[String, Row]): Unit = { | ||
| observedMetrics.forEach { | ||
| case (k, v) => builder.putObservedMetrics(k, mapper.writeValueAsString(v)) |
There was a problem hiding this comment.
nit: Let's add one line comment to mention why we choose to encode the row object with json.
gengliangwang
left a comment
There was a problem hiding this comment.
LGTM. Thanks for working on this during holiday!
|
done. Thank you for reviewing the code during the holiday :) @gengliangwang |
|
GA passed |
…ngQueryProgressWrapper` ### What changes were proposed in this pull request? Add Protobuf serializer for `StreamingQueryProgressWrapper ` ### Why are the changes needed? Support fast and compact serialization/deserialization for `StreamingQueryProgressWrapper` over RocksDB. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Add new UT - Manual test sql module with `LIVE_UI_LOCAL_STORE_DIR`, all test passed: ``` build/mvn clean install -DskipTests -pl sq/core -am export LIVE_UI_LOCAL_STORE_DIR=/tmp/spark-ui build/mvn clean install -pl sql/core ``` There 4 existing test suites classes use `StreamingQueryProgressWrapper`: - StateStoreCoordinatorSuite - StreamingQueryStatusListenerWithDiskStoreSuite - UISeleniumSuite - UISeleniumWithRocksDBBackendSuite Closes #39642 from LuciferYang/SPARK-41677-2. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: Gengliang Wang <gengliang@apache.org> (cherry picked from commit 7b93415) Signed-off-by: Gengliang Wang <gengliang@apache.org>
|
Thanks, merged to master & branch-3.4 |
|
Thanks @gengliangwang |
…ngQueryProgressWrapper` Add Protobuf serializer for `StreamingQueryProgressWrapper ` Support fast and compact serialization/deserialization for `StreamingQueryProgressWrapper` over RocksDB. No - Add new UT - Manual test sql module with `LIVE_UI_LOCAL_STORE_DIR`, all test passed: ``` build/mvn clean install -DskipTests -pl sq/core -am export LIVE_UI_LOCAL_STORE_DIR=/tmp/spark-ui build/mvn clean install -pl sql/core ``` There 4 existing test suites classes use `StreamingQueryProgressWrapper`: - StateStoreCoordinatorSuite - StreamingQueryStatusListenerWithDiskStoreSuite - UISeleniumSuite - UISeleniumWithRocksDBBackendSuite Closes apache#39642 from LuciferYang/SPARK-41677-2. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: Gengliang Wang <gengliang@apache.org> (cherry picked from commit 7b93415) Signed-off-by: Gengliang Wang <gengliang@apache.org>
…ngQueryProgressWrapper` Add Protobuf serializer for `StreamingQueryProgressWrapper ` Support fast and compact serialization/deserialization for `StreamingQueryProgressWrapper` over RocksDB. No - Add new UT - Manual test sql module with `LIVE_UI_LOCAL_STORE_DIR`, all test passed: ``` build/mvn clean install -DskipTests -pl sq/core -am export LIVE_UI_LOCAL_STORE_DIR=/tmp/spark-ui build/mvn clean install -pl sql/core ``` There 4 existing test suites classes use `StreamingQueryProgressWrapper`: - StateStoreCoordinatorSuite - StreamingQueryStatusListenerWithDiskStoreSuite - UISeleniumSuite - UISeleniumWithRocksDBBackendSuite Closes apache#39642 from LuciferYang/SPARK-41677-2. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: Gengliang Wang <gengliang@apache.org> (cherry picked from commit 7b93415) Signed-off-by: Gengliang Wang <gengliang@apache.org>
What changes were proposed in this pull request?
Add Protobuf serializer for
StreamingQueryProgressWrapperWhy are the changes needed?
Support fast and compact serialization/deserialization for
StreamingQueryProgressWrapperover RocksDB.Does this PR introduce any user-facing change?
No
How was this patch tested?
LIVE_UI_LOCAL_STORE_DIR, all test passed:There 4 existing test suites classes use
StreamingQueryProgressWrapper: