Skip to content

Conversation

@LuciferYang
Copy link
Contributor

@LuciferYang LuciferYang commented Dec 20, 2022

What changes were proposed in this pull request?

Add Protobuf serializer for SQLExecutionUIData

Why are the changes needed?

Support fast and compact serialization/deserialization for SQLExecutionUIData over RocksDB.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Add new UT

@LuciferYang LuciferYang marked this pull request as draft December 20, 2022 12:54
@LuciferYang LuciferYang changed the title [SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData [WIP][SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData Dec 20, 2022
}
}

message SQLPlanMetric {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Should we define it in the core module? Maybe it's better to put it in the sql module?

Copy link
Member

Choose a reason for hiding this comment

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

It is ok to define it in core module. There is "SQL" concept in core module anyway.
Can we skip shading protobuf in SQL module in this way?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If we define all proto messages in the core module, I think we can skip shading protobuf in SQL module, the generated message code can find the corresponding shaded+relocated dependencies in the core module

@LuciferYang
Copy link
Contributor Author

Wait some init work for sql module from #39164

@LuciferYang LuciferYang changed the title [WIP][SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData [SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData Dec 22, 2022
@LuciferYang LuciferYang changed the title [SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData [SPARK-41431][CORE][SQL][UI] Protobuf serializer for SQLExecutionUIData Dec 22, 2022
@LuciferYang LuciferYang marked this pull request as ready for review December 22, 2022 10:53
<groupId>org.apache.xbean</groupId>
<artifactId>xbean-asm9-shaded</artifactId>
</dependency>
<dependency>
Copy link
Member

Choose a reason for hiding this comment

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

This is not required, right?

Copy link
Contributor Author

@LuciferYang LuciferYang Dec 23, 2022

Choose a reason for hiding this comment

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

For sbt, it is not required:

[info] KVStoreProtobufSerializerSuite:
[info] - SQLExecutionUIData (121 milliseconds)
[info] Run completed in 1 second, 648 milliseconds.
[info] Total number of tests run: 1
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 293 s (04:53), completed 2022-12-23 9:41:14

But for maven, it is required now:

mvn clean test -pl sql/core -am -Dtest=none -DwildcardSuites=org.apache.spark.status.protobuf.sql.KVStoreProtobufSerializerSuite
[INFO] --- scala-maven-plugin:4.8.0:compile (scala-compile-first) @ spark-sql_2.12 ---
[INFO] Compiler bridge file: /Users/yangjie01/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.12-1.8.0-bin_2.12.17__52.0-1.8.0_20221110T195421.jar
[INFO] compiler plugin: BasicArtifact(com.github.ghik,silencer-plugin_2.12.17,1.7.10,null)
[INFO] compiling 576 Scala sources and 75 Java sources to /spark-source/sql/core/target/scala-2.12/classes ...
[ERROR] [Error] /spark-source/sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/SQLExecutionUIDataSerializer.scala:35: Class com.google.protobuf.GeneratedMessageV3 not found - continuing with a stub.
[ERROR] [Error] : Unable to locate class corresponding to inner class entry for Builder in owner com.google.protobuf.GeneratedMessageV3
[ERROR] [Error] /spark-source/sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/SQLExecutionUIDataSerializer.scala:43: Class com.google.protobuf.GeneratedMessageV3 not found - continuing with a stub.
[ERROR] [Error] /spark-source/sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/SQLExecutionUIDataSerializer.scala:55: value toByteArray is not a member of org.apache.spark.status.protobuf.StoreTypes.SQLExecutionUIData
[ERROR] [Error] : Unable to locate class corresponding to inner class entry for Builder in owner com.google.protobuf.GeneratedMessageV3
[ERROR] 5 errors found

during to sql module inherits protobuf-java 2.5.0 with compile scope from parent pom.xml without this change. We can verify this from the sql-module-dependency-tree.txt

Copy link
Contributor Author

Choose a reason for hiding this comment

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

#39164 (comment)

If the place we mentioned yesterday can be changed to 3.21.11, then we can remove this declaration

Copy link
Member

Choose a reason for hiding this comment

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

ok, do we need to shade it?

Copy link
Contributor Author

@LuciferYang LuciferYang Dec 23, 2022

Choose a reason for hiding this comment

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

Don't need shaded, I make a spark client with this pr for testing, and it can be parsed normally

EDIT: run TPCDSQueryBenchmark with spark-client and check ui

Copy link
Member

Choose a reason for hiding this comment

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

Hmmmm, did you test with both Maven and SBT? Then why core module requires shading..

Copy link
Contributor Author

@LuciferYang LuciferYang Dec 23, 2022

Choose a reason for hiding this comment

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

Yes, both MAVEN and SBT tested.

Personally, I think the core module needs shaded+relocation because we don't want protobuf version which Spark depends on to affect other third-party projects, just like connect&protobuf module. And the SQL module only uses the protobuf class generated in the core module, so it does not need to shaded + relocate again.

In fact, because Spark already uses the unified protobuf version, the connect and protobuf module only need relocation protobuf package to keep consistent with the relocation rules of the core module , rather than shaded protobuf-java again.

}
ui.stages.foreach(stageId => builder.addStages(stageId.toLong))
val metricValues = ui.metricValues
if (metricValues != null) {
Copy link
Contributor Author

@LuciferYang LuciferYang Dec 23, 2022

Choose a reason for hiding this comment

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

found a corner case. metricValues may be null, there will be an NPE if no null check

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@volatile var metricsValues: Map[Long, String] = null
// Just in case job end and execution end arrive out of order, keep track of how many
// end events arrived so that the listener can stop tracking the execution.
val endEvents = new AtomicInteger()
override protected def doUpdate(): Any = {
new SQLExecutionUIData(
executionId,
description,
details,
physicalPlanDescription,
modifiedConfigs,
metrics,
submissionTime,
completionTime,
errorMessage,
jobs,
stages,
metricsValues)

Or we can protect metricsValues is not null in line498 or line 517, but because I am not sure whether it will affect the existing scenario(like json), I think we can try in another pr

)
val bytes2 = serializer.serialize(input2)
val result2 = serializer.deserialize(bytes2, classOf[SQLExecutionUIData])
// input.metricValues is null, result.metricValues is also empty map.
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@gengliangwang For the collection type, the input is null, and after deserialization is empty collection, which may be different from json

Copy link
Member

Choose a reason for hiding this comment

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

Returning an empty collection seems safer. It won't affect the UI page anyway. Thanks for noticing it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK

@LuciferYang
Copy link
Contributor Author

@gengliangwang A question unrelated to the pr :

Should we clean up the spark.ui.store.path directory when SparkContext.stop()? Otherwise, 1.The disk space occupied by the spark.ui.store.path directory will continue to grow. 2. When submitting new App and reusing the spark.ui.store.path directory, we will see the content related to the previous App, which is a bit strange

@gengliangwang
Copy link
Member

Should we clean up the spark.ui.store.path directory when SparkContext.stop()? Otherwise, 1.The disk space occupied by the spark.ui.store.path directory will continue to grow. 2. When submitting new App and reusing the spark.ui.store.path directory, we will see the content related to the previous App, which is a bit strange

Good question. We can have a configuration for this one. By default, we delete the RocksDB instance.
Some users may want to see the history, or the file can be used for history server.

@LuciferYang
Copy link
Contributor Author

LuciferYang commented Dec 23, 2022

Should we clean up the spark.ui.store.path directory when SparkContext.stop()? Otherwise, 1.The disk space occupied by the spark.ui.store.path directory will continue to grow. 2. When submitting new App and reusing the spark.ui.store.path directory, we will see the content related to the previous App, which is a bit strange

Good question. We can have a configuration for this one. By default, we delete the RocksDB instance. Some users may want to see the history, or the file can be used for history server.

OK, let me file a new jira first to tracking this : SPARK-41694

@gengliangwang
Copy link
Member

Merging to master

@LuciferYang
Copy link
Contributor Author

thanks @gengliangwang

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.

2 participants