stefankandic commented on code in PR #46280:
URL: https://github.com/apache/spark/pull/46280#discussion_r1601881695


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala:
##########
@@ -63,7 +66,60 @@ case class StructField(
     ("name" -> name) ~
       ("type" -> dataType.jsonValue) ~
       ("nullable" -> nullable) ~
-      ("metadata" -> metadata.jsonValue)
+      ("metadata" -> metadataJson)
+  }
+
+  private def metadataJson: JValue = {
+    val metadataJsonValue = metadata.jsonValue
+    metadataJsonValue match {
+      case JObject(fields) if collationMetadata.nonEmpty =>
+        val collationFields = collationMetadata.map(kv => kv._1 -> 
JString(kv._2)).toList
+        JObject(fields :+ (DataType.COLLATIONS_METADATA_KEY -> 
JObject(collationFields)))
+
+      case _ => metadataJsonValue
+    }
+  }
+
+  /** Map of field path to collation name. */
+  private lazy val collationMetadata: mutable.Map[String, String] = {
+    val fieldToCollationMap = mutable.Map[String, String]()
+
+    def visitRecursively(dt: DataType, path: String): Unit = dt match {
+      case at: ArrayType =>
+        processDataType(at.elementType, path + ".element")
+
+      case mt: MapType =>
+        processDataType(mt.keyType, path + ".key")
+        processDataType(mt.valueType, path + ".value")
+
+      case st: StringType if isCollatedString(st) =>
+        fieldToCollationMap(path) = collationName(st)
+
+      case _ =>
+    }
+
+    def processDataType(dt: DataType, path: String): Unit = {
+      if (isCollatedString(dt)) {
+        fieldToCollationMap(path) = collationName(dt)
+      } else {
+        visitRecursively(dt, path)
+      }
+    }
+
+    visitRecursively(dataType, name)

Review Comment:
   Each column has separate metadata so map and string can never clash -> here 
is how it would look like in json:
   
   ```json
   {
     "type": "struct",
     "fields": [
       {
         "name": "c1.key",
         "type": "string",
         "nullable": true,
         "metadata": {
           "__COLLATIONS": {
             "c1.key": "ICU.UNICODE_CI"
           }
         }
       },
       {
         "name": "c1",
         "type": {
           "type": "map",
           "keyType": "string",
           "valueType": "string",
           "valueContainsNull": true
         },
         "nullable": true,
         "metadata": {
           "__COLLATIONS": {
             "c1.key": "ICU.UNICODE"
           }
         }
       }
     ]
   }
   ```
   
   "c1.key" will mean different things in each column - for the first it's just 
the entire column and for the second it is the key of the map



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to