You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "yhosny (via GitHub)" <gi...@apache.org> on 2024/03/11 20:22:17 UTC

[PR] [SPARK-47345] [SQL]: Xml functions suite [spark]

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

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Convert JsonFunctiosnSuite.scala to XML equivalent. Note that XML doesn’t implement all json functions like json_tuple, get_json_object, etc.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Improve unit test coverage.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   Unit tests.
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   No.


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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala:
##########
@@ -0,0 +1,480 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+class XmlFunctionsSuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  test("from_xml") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (timestampFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015 18:00</time></ROW>""").toDS()
+    val schema = new StructType().add("time", TimestampType)
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+  }
+
+  test("from_xml with option (rowTag)") {
+    val df = Seq("""<foo><a>1</a></foo>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+    val options = Map("rowTag" -> "foo").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (dateFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015</time></ROW>""").toDS()
+    val schema = new StructType().add("time", DateType)
+    val options = Map("dateFormat" -> "dd/MM/yyyy").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Date.valueOf("2015-08-26"))))
+  }
+
+  test("from_xml missing columns") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("b", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml invalid xml") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml - xml doesn't conform to the array type") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(df.select(from_xml($"value", schema)), Row(Row(null)))
+  }
+
+  test("from_xml array support") {
+    val df = Seq(s"""<ROW> <a>1</a> <a>2</a> </ROW>""".stripMargin).toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(Array(1, 2))))
+  }
+
+  test("from_xml uses DDL strings for defining a schema - java") {
+    val df = Seq("""<ROW> <a>1</a> <b>haa</b> </ROW>""").toDS()
+    checkAnswer(
+      df.select(from_xml($"value", "a INT, b STRING", new java.util.HashMap[String, String]())),
+      Row(Row(1, "haa")) :: Nil)
+  }
+
+  test("to_xml - struct") {
+    val schema = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val data = Seq(Row(1))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a")),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (timestampFormat)") {
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
+    val schema = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (dateFormat)") {
+    val options = Map("dateFormat" -> "dd/MM/yyyy")
+    val schema = StructType(StructField("a", DateType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Date.valueOf("2015-08-26")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("roundtrip in to_xml and from_xml - struct") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1, 2, 3))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", IntegerType)
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("roundtrip in to_xml and from_xml - array") {
+    val schemaOne = StructType(StructField("a", ArrayType(IntegerType), nullable = false) :: Nil)
+    val dataOne = Seq(Row(Array(1, 2, 3)))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", ArrayType(IntegerType))
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("Support to_xml in SQL") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer (
+      df1.selectExpr("to_xml(a)"),
+      Row(xml) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    val schema2 = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val dataTwo = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df2 = spark.createDataFrame(spark.sparkContext.parallelize(dataTwo), schema2)
+      .withColumn("a", struct($"a"))
+    checkAnswer(
+      df2.selectExpr("to_xml(a, map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(xml2) :: Nil)
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "to_xml(a, named_struct('a', 1))",
+        start = 0,
+        stop = 30
+      )
+    )
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "to_xml(a, map('a', 1))",
+        start = 0,
+        stop = 21
+      )
+    )
+  }
+
+  test("Support from_xml in SQL") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val df1 = Seq(xml).toDS()
+    checkAnswer(
+      df1.selectExpr("from_xml(value, 'a INT')"),
+      Row(Row(1)) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <c0>a</c0>
+          |    <c1>1</c1>
+          |    <c2>
+          |        <c20>
+          |            3.8
+          |        </c20>
+          |        <c21>
+          |            8
+          |        </c21>
+          |    </c2>
+          |</ROW>""".stripMargin
+    val df2 = Seq(xml2).toDS()
+    checkAnswer(
+      df2.selectExpr("from_xml(value, 'c0 STRING, c1 INT, c2 STRUCT<c20: DOUBLE, c21: INT>')"),
+      Row(Row("a", 1, Row(3.8, 8))) :: Nil)
+
+    val xml3 =
+      s"""|<ROW>
+          |    <time>26/08/2015 18:00</time>
+          |</ROW>""".stripMargin
+    val df3 = Seq(xml3).toDS()
+    checkAnswer(
+      df3.selectExpr(
+        "from_xml(value, 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 1)")
+      },
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"1\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 1)",
+        start = 0,
+        stop = 17
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("""from_xml(value, 'time InvalidType')""")
+      },
+      errorClass = "PARSE_SYNTAX_ERROR",
+      sqlState = "42601",
+      parameters = Map(
+        "error" -> "'InvalidType'",
+        "hint" -> ": extra input 'InvalidType'"
+      ),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time InvalidType')",
+        start = 0,
+        stop = 34
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', named_struct('a', 1))",
+        start = 0,
+        stop = 54
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', map('a', 1))",
+        start = 0,
+        stop = 45
+      )
+    )
+  }
+
+  test("infers schemas of xml strings and pass them to from_xml") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |    <a>3</a>
+          |</ROW>""".stripMargin
+    val in = Seq(xml).toDS()
+    val out = in.select(from_xml($"value", schema_of_xml(xml)) as "parsed")
+    val expected = StructType(StructField(
+      "parsed",
+      StructType(StructField(
+        "a",
+        ArrayType(LongType, true), true) :: Nil),
+      true) :: Nil)
+
+    assert(out.schema == expected)
+  }
+
+  test("infers schemas using options") {
+    val df = spark.range(1)
+      .select(schema_of_xml(lit("<ROW><a>1</a></ROW>"),
+        Map("allowUnquotedFieldNames" -> "true").asJava))
+    checkAnswer(df, Seq(Row("STRUCT<a: BIGINT>")))
+  }
+
+  test("from_xml invalid xml - check modes") {
+    withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") {
+      val schema = new StructType()
+        .add("a", IntegerType)
+        .add("b", IntegerType)
+        .add("_unparsed", StringType)
+      val badRec = """<ROW><a>1<b><2></b></ROW>"""
+      val df = Seq(badRec, """<ROW><a>2</a><b>12</b></ROW>""").toDS()
+
+      checkAnswer(
+        df.select(from_xml($"value", schema, Map("mode" -> "PERMISSIVE").asJava)),
+        Row(Row(null, null, badRec)) :: Row(Row(2, 12, null)) :: Nil)
+    }
+  }
+
+  test("corrupt record column in the middle") {
+    val schema = new StructType()
+      .add("a", IntegerType)
+      .add("_unparsed", StringType)
+      .add("b", IntegerType)
+    val badRec = """<ROW><a>1</a><b>2</ROW>"""
+    val df = Seq(badRec, """<ROW><a>1</a><b>12</b></ROW>""").toDS()
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, Map("columnNameOfCorruptRecord" -> "_unparsed").asJava)),
+      Row(Row(null, badRec, null)) :: Row(Row(1, null, null)) :: Nil)
+  }
+
+  test("parse timestamps with locale") {
+    Seq("en-US", "ko-KR", "zh-CN", "ru-RU").foreach { langTag =>
+      val locale = Locale.forLanguageTag(langTag)
+      val ts = new SimpleDateFormat("dd/MM/yyyy HH:mm").parse("06/11/2018 18:00")
+      val timestampFormat = "dd MMM yyyy HH:mm"
+      val sdf = new SimpleDateFormat(timestampFormat, locale)
+      val input = Seq(s"""<ROW><time>${sdf.format(ts)}</time></ROW>""").toDS()
+      val options = Map("timestampFormat" -> timestampFormat, "locale" -> langTag).asJava
+      val df = input.select(from_xml($"value", "time timestamp", options))
+
+      checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0"))))
+    }
+  }
+
+  test("from_xml - timestamp in micros") {
+    val df = Seq("""<ROW><time>1970-01-01T00:00:00.123456</time></ROW>""").toDS()
+    val schema = new StructType().add("time", TimestampType)
+    val options = Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSSSS").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456"))))
+  }
+
+  test("to_xml - timestamp in micros") {
+    val s = "2019-11-18 11:56:00.123456"
+    val xml =
+      s"""|<ROW>
+          |    <t>$s</t>
+          |</ROW>""".stripMargin
+    val df = Seq(java.sql.Timestamp.valueOf(s)).toDF("t").select(
+      to_xml(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS").asJava))
+    checkAnswer(df, Row(xml))
+  }
+
+  test("support foldable schema by from_xml") {
+    val options = Map[String, String]().asJava
+    val schema = regexp_replace(lit("dpt_org_id INT, dpt_org_city STRING"), "dpt_org_", "")
+    checkAnswer(
+      Seq("""<ROW><id>1</id><city>Moscow</city></ROW>""").toDS()
+        .select(from_xml($"value", schema, options)),
+      Row(Row(1, "Moscow")))
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        Seq(("""<ROW><i>1</i></ROW>""", "i int")).toDF("xml", "schema")
+          .select(from_xml($"xml", $"schema", options)).collect()
+      },
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"schema\""),
+      context = ExpectedContext(fragment = "from_xml", getCurrentClassCallSitePattern)
+    )
+  }
+
+  test("schema_of_xml - infers the schema of foldable JSON string") {

Review Comment:
   Made a followup: https://github.com/apache/spark/pull/45577



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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

Posted by "sandip-db (via GitHub)" <gi...@apache.org>.
sandip-db commented on code in PR #45466:
URL: https://github.com/apache/spark/pull/45466#discussion_r1529511772


##########
sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala:
##########
@@ -0,0 +1,480 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+class XmlFunctionsSuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  test("from_xml") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (timestampFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015 18:00</time></ROW>""").toDS()
+    val schema = new StructType().add("time", TimestampType)
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+  }
+
+  test("from_xml with option (rowTag)") {
+    val df = Seq("""<foo><a>1</a></foo>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+    val options = Map("rowTag" -> "foo").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (dateFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015</time></ROW>""").toDS()
+    val schema = new StructType().add("time", DateType)
+    val options = Map("dateFormat" -> "dd/MM/yyyy").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Date.valueOf("2015-08-26"))))
+  }
+
+  test("from_xml missing columns") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("b", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml invalid xml") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml - xml doesn't conform to the array type") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(df.select(from_xml($"value", schema)), Row(Row(null)))
+  }
+
+  test("from_xml array support") {
+    val df = Seq(s"""<ROW> <a>1</a> <a>2</a> </ROW>""".stripMargin).toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(Array(1, 2))))
+  }
+
+  test("from_xml uses DDL strings for defining a schema - java") {
+    val df = Seq("""<ROW> <a>1</a> <b>haa</b> </ROW>""").toDS()
+    checkAnswer(
+      df.select(from_xml($"value", "a INT, b STRING", new java.util.HashMap[String, String]())),
+      Row(Row(1, "haa")) :: Nil)
+  }
+
+  test("to_xml - struct") {
+    val schema = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val data = Seq(Row(1))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a")),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (timestampFormat)") {
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
+    val schema = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (dateFormat)") {
+    val options = Map("dateFormat" -> "dd/MM/yyyy")
+    val schema = StructType(StructField("a", DateType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Date.valueOf("2015-08-26")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("roundtrip in to_xml and from_xml - struct") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1, 2, 3))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", IntegerType)
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("roundtrip in to_xml and from_xml - array") {
+    val schemaOne = StructType(StructField("a", ArrayType(IntegerType), nullable = false) :: Nil)
+    val dataOne = Seq(Row(Array(1, 2, 3)))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", ArrayType(IntegerType))
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("Support to_xml in SQL") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer (
+      df1.selectExpr("to_xml(a)"),
+      Row(xml) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    val schema2 = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val dataTwo = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df2 = spark.createDataFrame(spark.sparkContext.parallelize(dataTwo), schema2)
+      .withColumn("a", struct($"a"))
+    checkAnswer(
+      df2.selectExpr("to_xml(a, map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(xml2) :: Nil)
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "to_xml(a, named_struct('a', 1))",
+        start = 0,
+        stop = 30
+      )
+    )
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "to_xml(a, map('a', 1))",
+        start = 0,
+        stop = 21
+      )
+    )
+  }
+
+  test("Support from_xml in SQL") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val df1 = Seq(xml).toDS()
+    checkAnswer(
+      df1.selectExpr("from_xml(value, 'a INT')"),
+      Row(Row(1)) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <c0>a</c0>
+          |    <c1>1</c1>
+          |    <c2>
+          |        <c20>
+          |            3.8
+          |        </c20>
+          |        <c21>
+          |            8
+          |        </c21>
+          |    </c2>
+          |</ROW>""".stripMargin
+    val df2 = Seq(xml2).toDS()
+    checkAnswer(
+      df2.selectExpr("from_xml(value, 'c0 STRING, c1 INT, c2 STRUCT<c20: DOUBLE, c21: INT>')"),
+      Row(Row("a", 1, Row(3.8, 8))) :: Nil)
+
+    val xml3 =
+      s"""|<ROW>
+          |    <time>26/08/2015 18:00</time>
+          |</ROW>""".stripMargin
+    val df3 = Seq(xml3).toDS()
+    checkAnswer(
+      df3.selectExpr(
+        "from_xml(value, 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 1)")
+      },
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"1\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 1)",
+        start = 0,
+        stop = 17
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("""from_xml(value, 'time InvalidType')""")
+      },
+      errorClass = "PARSE_SYNTAX_ERROR",
+      sqlState = "42601",
+      parameters = Map(
+        "error" -> "'InvalidType'",
+        "hint" -> ": extra input 'InvalidType'"
+      ),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time InvalidType')",
+        start = 0,
+        stop = 34
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', named_struct('a', 1))",
+        start = 0,
+        stop = 54
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', map('a', 1))",
+        start = 0,
+        stop = 45
+      )
+    )
+  }
+
+  test("infers schemas of xml strings and pass them to from_xml") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |    <a>3</a>
+          |</ROW>""".stripMargin
+    val in = Seq(xml).toDS()
+    val out = in.select(from_xml($"value", schema_of_xml(xml)) as "parsed")
+    val expected = StructType(StructField(
+      "parsed",
+      StructType(StructField(
+        "a",
+        ArrayType(LongType, true), true) :: Nil),
+      true) :: Nil)
+
+    assert(out.schema == expected)
+  }
+
+  test("infers schemas using options") {
+    val df = spark.range(1)
+      .select(schema_of_xml(lit("<ROW><a>1</a></ROW>"),
+        Map("allowUnquotedFieldNames" -> "true").asJava))
+    checkAnswer(df, Seq(Row("STRUCT<a: BIGINT>")))
+  }
+
+  test("from_xml invalid xml - check modes") {
+    withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") {
+      val schema = new StructType()
+        .add("a", IntegerType)
+        .add("b", IntegerType)
+        .add("_unparsed", StringType)
+      val badRec = """<ROW><a>1<b><2></b></ROW>"""
+      val df = Seq(badRec, """<ROW><a>2</a><b>12</b></ROW>""").toDS()
+
+      checkAnswer(
+        df.select(from_xml($"value", schema, Map("mode" -> "PERMISSIVE").asJava)),
+        Row(Row(null, null, badRec)) :: Row(Row(2, 12, null)) :: Nil)
+    }
+  }
+
+  test("corrupt record column in the middle") {
+    val schema = new StructType()
+      .add("a", IntegerType)
+      .add("_unparsed", StringType)
+      .add("b", IntegerType)
+    val badRec = """<ROW><a>1</a><b>2</ROW>"""
+    val df = Seq(badRec, """<ROW><a>1</a><b>12</b></ROW>""").toDS()
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, Map("columnNameOfCorruptRecord" -> "_unparsed").asJava)),
+      Row(Row(null, badRec, null)) :: Row(Row(1, null, null)) :: Nil)
+  }
+
+  test("parse timestamps with locale") {
+    Seq("en-US", "ko-KR", "zh-CN", "ru-RU").foreach { langTag =>
+      val locale = Locale.forLanguageTag(langTag)
+      val ts = new SimpleDateFormat("dd/MM/yyyy HH:mm").parse("06/11/2018 18:00")
+      val timestampFormat = "dd MMM yyyy HH:mm"
+      val sdf = new SimpleDateFormat(timestampFormat, locale)
+      val input = Seq(s"""<ROW><time>${sdf.format(ts)}</time></ROW>""").toDS()
+      val options = Map("timestampFormat" -> timestampFormat, "locale" -> langTag).asJava
+      val df = input.select(from_xml($"value", "time timestamp", options))
+
+      checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0"))))
+    }
+  }
+
+  test("from_xml - timestamp in micros") {
+    val df = Seq("""<ROW><time>1970-01-01T00:00:00.123456</time></ROW>""").toDS()
+    val schema = new StructType().add("time", TimestampType)
+    val options = Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSSSS").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456"))))
+  }
+
+  test("to_xml - timestamp in micros") {
+    val s = "2019-11-18 11:56:00.123456"
+    val xml =
+      s"""|<ROW>
+          |    <t>$s</t>
+          |</ROW>""".stripMargin
+    val df = Seq(java.sql.Timestamp.valueOf(s)).toDF("t").select(
+      to_xml(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS").asJava))
+    checkAnswer(df, Row(xml))
+  }
+
+  test("support foldable schema by from_xml") {
+    val options = Map[String, String]().asJava
+    val schema = regexp_replace(lit("dpt_org_id INT, dpt_org_city STRING"), "dpt_org_", "")
+    checkAnswer(
+      Seq("""<ROW><id>1</id><city>Moscow</city></ROW>""").toDS()
+        .select(from_xml($"value", schema, options)),
+      Row(Row(1, "Moscow")))
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        Seq(("""<ROW><i>1</i></ROW>""", "i int")).toDF("xml", "schema")
+          .select(from_xml($"xml", $"schema", options)).collect()
+      },
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"schema\""),
+      context = ExpectedContext(fragment = "from_xml", getCurrentClassCallSitePattern)
+    )
+  }
+
+  test("schema_of_xml - infers the schema of foldable JSON string") {

Review Comment:
   JSON -> XML



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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala:
##########
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+class XmlFunctionsSuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+

Review Comment:
   remove this line



##########
sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala:
##########
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+class XmlFunctionsSuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+
+  test("from_xml") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (timestampFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015 18:00</time></ROW>""").toDS()
+    val schema = new StructType().add("time", TimestampType)
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+  }
+
+  test("from_xml with option (rowTag)") {
+    val df = Seq("""<foo><a>1</a></foo>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+    val options = Map("rowTag" -> "foo").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(1)) :: Nil)
+  }
+
+  test("from_xml with option (dateFormat)") {
+    val df = Seq("""<ROW><time>26/08/2015</time></ROW>""").toDS()
+    val schema = new StructType().add("time", DateType)
+    val options = Map("dateFormat" -> "dd/MM/yyyy").asJava
+
+    checkAnswer(
+      df.select(from_xml($"value", schema, options)),
+      Row(Row(java.sql.Date.valueOf("2015-08-26"))))
+  }
+
+  test("from_xml missing columns") {
+    val df = Seq("""<ROW><a>1</a></ROW>""").toDS()
+    val schema = new StructType().add("b", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml invalid xml") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = new StructType().add("a", IntegerType)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(null)) :: Nil)
+  }
+
+  test("from_xml - xml doesn't conform to the array type") {
+    val df = Seq("""<ROW><a>1</ROW>""").toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(df.select(from_xml($"value", schema)), Row(Row(null)))
+  }
+
+  test("from_xml array support") {
+    val df = Seq(s"""<ROW> <a>1</a> <a>2</a> </ROW>""".stripMargin).toDS()
+    val schema = StructType(StructField("a", ArrayType(IntegerType)) :: Nil)
+
+    checkAnswer(
+      df.select(from_xml($"value", schema)),
+      Row(Row(Array(1, 2))))
+  }
+
+  test("from_xml uses DDL strings for defining a schema - java") {
+    val df = Seq("""<ROW> <a>1</a> <b>haa</b> </ROW>""").toDS()
+    checkAnswer(
+      df.select(from_xml($"value", "a INT, b STRING", new java.util.HashMap[String, String]())),
+      Row(Row(1, "haa")) :: Nil)
+  }
+
+  test("to_xml - struct") {
+    val schema = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val data = Seq(Row(1))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a")),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (timestampFormat)") {
+    val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
+    val schema = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("to_xml with option (dateFormat)") {
+    val options = Map("dateFormat" -> "dd/MM/yyyy")
+    val schema = StructType(StructField("a", DateType, nullable = false) :: Nil)
+    val data = Seq(Row(java.sql.Date.valueOf("2015-08-26")))
+    val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
+      .withColumn("a", struct($"a"))
+
+    val expected =
+      s"""|<ROW>
+          |    <a>26/08/2015</a>
+          |</ROW>""".stripMargin
+    checkAnswer(
+      df.select(to_xml($"a", options.asJava)),
+      Row(expected) :: Nil)
+  }
+
+  test("roundtrip in to_xml and from_xml - struct") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1, 2, 3))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", IntegerType)
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("roundtrip in to_xml and from_xml - array") {
+    val schemaOne = StructType(StructField("a", ArrayType(IntegerType), nullable = false) :: Nil)
+    val dataOne = Seq(Row(Array(1, 2, 3)))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val readBackOne = df1.select(to_xml($"a").as("xml"))
+      .select(from_xml($"xml", schemaOne).as("a"))
+    checkAnswer(df1, readBackOne)
+
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |</ROW>""".stripMargin
+    val schemaTwo = new StructType().add("a", ArrayType(IntegerType))
+    val dfTwo = Seq(Some(xml), None).toDF("xml")
+    val readBackTwo = dfTwo.select(from_xml($"xml", schemaTwo).as("struct"))
+      .select(to_xml($"struct").as("xml"))
+    checkAnswer(dfTwo, readBackTwo)
+  }
+
+  test("Support to_xml in SQL") {
+    val schemaOne = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
+    val dataOne = Seq(Row(1))
+    val df1 = spark.createDataFrame(spark.sparkContext.parallelize(dataOne), schemaOne)
+      .withColumn("a", struct($"a"))
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    checkAnswer (
+      df1.selectExpr("to_xml(a)"),
+      Row(xml) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <a>26/08/2015 18:00</a>
+          |</ROW>""".stripMargin
+    val schema2 = StructType(StructField("a", TimestampType, nullable = false) :: Nil)
+    val dataTwo = Seq(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))
+    val df2 = spark.createDataFrame(spark.sparkContext.parallelize(dataTwo), schema2)
+      .withColumn("a", struct($"a"))
+    checkAnswer(
+      df2.selectExpr("to_xml(a, map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(xml2) :: Nil)
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "to_xml(a, named_struct('a', 1))",
+        start = 0,
+        stop = 30
+      )
+    )
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df2.selectExpr("to_xml(a, map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "to_xml(a, map('a', 1))",
+        start = 0,
+        stop = 21
+      )
+    )
+  }
+
+  test("Support from_xml in SQL") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |</ROW>""".stripMargin
+    val df1 = Seq(xml).toDS()
+    checkAnswer(
+      df1.selectExpr("from_xml(value, 'a INT')"),
+      Row(Row(1)) :: Nil)
+
+    val xml2 =
+      s"""|<ROW>
+          |    <c0>a</c0>
+          |    <c1>1</c1>
+          |    <c2>
+          |        <c20>
+          |            3.8
+          |        </c20>
+          |        <c21>
+          |            8
+          |        </c21>
+          |    </c2>
+          |</ROW>""".stripMargin
+    val df2 = Seq(xml2).toDS()
+    checkAnswer(
+      df2.selectExpr("from_xml(value, 'c0 STRING, c1 INT, c2 STRUCT<c20: DOUBLE, c21: INT>')"),
+      Row(Row("a", 1, Row(3.8, 8))) :: Nil)
+
+    val xml3 =
+      s"""|<ROW>
+          |    <time>26/08/2015 18:00</time>
+          |</ROW>""".stripMargin
+    val df3 = Seq(xml3).toDS()
+    checkAnswer(
+      df3.selectExpr(
+        "from_xml(value, 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy HH:mm'))"),
+      Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0"))))
+
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 1)")
+      },
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"1\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 1)",
+        start = 0,
+        stop = 17
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("""from_xml(value, 'time InvalidType')""")
+      },
+      errorClass = "PARSE_SYNTAX_ERROR",
+      sqlState = "42601",
+      parameters = Map(
+        "error" -> "'InvalidType'",
+        "hint" -> ": extra input 'InvalidType'"
+      ),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time InvalidType')",
+        start = 0,
+        stop = 34
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', named_struct('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_MAP_FUNCTION",
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', named_struct('a', 1))",
+        start = 0,
+        stop = 54
+      )
+    )
+    checkError(
+      exception = intercept[AnalysisException] {
+        df3.selectExpr("from_xml(value, 'time Timestamp', map('a', 1))")
+      },
+      errorClass = "INVALID_OPTIONS.NON_STRING_TYPE",
+      parameters = Map("mapType" -> "\"MAP<STRING, INT>\""),
+      context = ExpectedContext(
+        fragment = "from_xml(value, 'time Timestamp', map('a', 1))",
+        start = 0,
+        stop = 45
+      )
+    )
+  }
+
+  test("infers schemas of xml strings and pass them to from_xml") {
+    val xml =
+      s"""|<ROW>
+          |    <a>1</a>
+          |    <a>2</a>
+          |    <a>3</a>
+          |</ROW>""".stripMargin
+    val in = Seq(xml).toDS()
+    val out = in.select(from_xml($"value", schema_of_xml(xml)) as "parsed")
+    val expected = StructType(StructField(
+      "parsed",
+      StructType(StructField(
+        "a",
+        ArrayType(LongType, true), true) :: Nil),
+      true) :: Nil)
+
+    assert(out.schema == expected)
+  }
+
+  test("infers schemas using options") {
+    val df = spark.range(1)
+      .select(schema_of_xml(lit("<ROW><a>1</a></ROW>"),
+              Map("allowUnquotedFieldNames" -> "true").asJava))

Review Comment:
   please, fix indentation 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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

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

   I have checked the new test suite locally:
   ```
   $ build/sbt "sql/test:testOnly *XmlFunctionsSuite"
   [info] XmlFunctionsSuite:
   22:50:44.233 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   [info] - from_xml (5 seconds, 317 milliseconds)
   [info] - from_xml with option (timestampFormat) (382 milliseconds)
   [info] - from_xml with option (rowTag) (153 milliseconds)
   [info] - from_xml with option (dateFormat) (180 milliseconds)
   ...
   [info] All tests passed.
   ```
   
   +1, LGTM. Merging to master.
   Thank you, @yhosny and @zhengruifeng for review.


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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

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

   add `[TESTS]` since it only adds new tests


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


Re: [PR] [SPARK-47345][SQL][TESTS] Xml functions suite [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #45466: [SPARK-47345][SQL][TESTS] Xml functions suite
URL: https://github.com/apache/spark/pull/45466


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