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

[GitHub] [spark] SandishKumarHN commented on a diff in pull request #40983: [SPARK-43312][PROTOBUF] Option to convert Any fields into JSON

SandishKumarHN commented on code in PR #40983:
URL: https://github.com/apache/spark/pull/40983#discussion_r1179904542


##########
connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDeserializer.scala:
##########
@@ -70,6 +69,15 @@ private[sql] class ProtobufDeserializer(
 
   def deserialize(data: Message): Option[InternalRow] = converter(data)
 
+  // JsonFormatter use to convert Any fields (if the option is enabled)
+  // This keeps original field names and does not include any extra whitespace in JSON.
+  // This is used to convert Any fields to json (if configured in Protobuf options).
+  // If the runtime type for Any field is not found in the registry, it throws an exception.
+  private val jsonPrinter = JsonFormat.printer
+    .omittingInsignificantWhitespace()
+    .preservingProtoFieldNames()

Review Comment:
   @rangadi Are the names of Protobuf fields preserved for other types? I don't see any explicit use of this for other types.
   
   



##########
connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala:
##########
@@ -63,7 +64,16 @@ private[protobuf] case class ProtobufDataToCatalyst(
   @transient private lazy val fieldsNumbers =
     messageDescriptor.getFields.asScala.map(f => f.getNumber).toSet
 
-  @transient private lazy val deserializer = new ProtobufDeserializer(messageDescriptor, dataType)
+  @transient private lazy val deserializer = {
+    val typeRegistry = descFilePath match {
+      case Some(path) if protobufOptions.convertAnyFieldsToJson =>
+        ProtobufUtils.buildTypeRegistry(path) // This loads all the messages in the file.
+      case None if protobufOptions.convertAnyFieldsToJson =>
+        ProtobufUtils.buildTypeRegistry(messageDescriptor) // Loads only connected messages.
+      case _ => TypeRegistry.getEmptyTypeRegistry // Default. Json conversion is not enabled.
+    }
+    new ProtobufDeserializer(messageDescriptor, dataType, typeRegistry = typeRegistry)
+  }

Review Comment:
   @rangadi Is it possible to replace `buildDescriptor(descriptor, messageName)` with `ProtobufUtils.buildTypeRegistry(path)` ? and use `typeRegistry.find(messageName)` 



-- 
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