You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "grundprinzip (via GitHub)" <gi...@apache.org> on 2023/08/23 10:59:16 UTC

[GitHub] [spark] grundprinzip opened a new pull request, #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

grundprinzip opened a new pull request, #42630:
URL: https://github.com/apache/spark/pull/42630

   ### What changes were proposed in this pull request?
   This patch fixes an issue with the serialization of the Spark Connect proto messages during the event notifications. 
   
   Without this patch upon receiving Spark Connect events, the Spark log will be full of the following messages:
   
   ```
   23/08/23 08:55:52 WARN DBCEventLoggingListener: Error in writing the event to log
   com.fasterxml.jackson.databind.exc.InvalidDefinitionException: Direct self-reference leading to cycle (through reference chain: org.apache.spark.sql.connect.service.SparkListenerConnectOperationStarted["planRequest"]->org.apache.spark.connect.proto.ExecutePlanRequest["unknownFields"]->com.google.protobuf.UnknownFieldSet["defaultInstanceForType"])
   	at com.fasterxml.jackson.databind.exc.InvalidDefinitionException.from(InvalidDefinitionException.java:77)
   	at com.fasterxml.jackson.databind.SerializerProvider.reportBadDefinition(SerializerProvider.java:1308)
   	at com.fasterxml.jackson.databind.ser.BeanPropertyWriter._handleSelfReference(BeanPropertyWriter.java:948)
   	at com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:726)
   	at com.fasterxml.jackson.databind.ser.std.BeanSerializerBase.serializeFields(BeanSerializerBase.java:772)
   	at com.fasterxml.jackson.databind.ser.BeanSerializer.serialize(BeanSerializer.java:178)
   ```
   
   The reason is that the JSON serializer cannot properly deal with recursive nested structures and for that reason, a custom JSON serializer is added that uses the built-in protobuf serialization.
   
   ### Why are the changes needed?
   Stability / Compatibility
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   Existing UT


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #42630:
URL: https://github.com/apache/spark/pull/42630#discussion_r1303055899


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala:
##########
@@ -27,6 +31,14 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.connect.common.ProtoUtils
 import org.apache.spark.util.{Clock, Utils}
 
+private[sql] class ProtobufSerializer extends JsonSerializer[Message] {

Review Comment:
   Can [sql] be removed?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala:
##########
@@ -27,6 +31,14 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.connect.common.ProtoUtils
 import org.apache.spark.util.{Clock, Utils}
 
+private[sql] class ProtobufSerializer extends JsonSerializer[Message] {

Review Comment:
   Can `[sql]` be removed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "grundprinzip (via GitHub)" <gi...@apache.org>.
grundprinzip commented on code in PR #42630:
URL: https://github.com/apache/spark/pull/42630#discussion_r1303078024


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala:
##########
@@ -27,6 +31,14 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.connect.common.ProtoUtils
 import org.apache.spark.util.{Clock, Utils}
 
+private[sql] class ProtobufSerializer extends JsonSerializer[Message] {

Review Comment:
   I can remove the `[sql]`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #42630:
URL: https://github.com/apache/spark/pull/42630#issuecomment-1690008669

   @grundprinzip Sorry, I didn't find a way to validate this patch. Is there any chance you could add a new test case or describe how to verify it manually?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "grundprinzip (via GitHub)" <gi...@apache.org>.
grundprinzip commented on PR #42630:
URL: https://github.com/apache/spark/pull/42630#issuecomment-1690210205

   Closing in favor of #42550


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip closed pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "grundprinzip (via GitHub)" <gi...@apache.org>.
grundprinzip closed pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener
URL: https://github.com/apache/spark/pull/42630


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] jdesjean commented on pull request #42630: [SPARK-44931] Fix JSON Serialization of Spark Connect protos for Event Listener

Posted by "jdesjean (via GitHub)" <gi...@apache.org>.
jdesjean commented on PR #42630:
URL: https://github.com/apache/spark/pull/42630#issuecomment-1690121178

   I had a [separate PR](https://github.com/apache/spark/pull/42550) that ignored the field from serde. I'm open to either approach. The other PR has some additional testing that could be used here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org