From 4f25df1dc25cc4f002107821cc67e35c1fe0e42c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 9 Aug 2023 19:16:37 +0800 Subject: [PATCH 01/12] [SPARK-43979][SQL][FOLLOWUP] transformUpWithNewOutput` should only be used with new outputs ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/41475 . It's risky to use `transformUpWithNewOutput` with existing attribute ids. If the plan contains duplicated attribute ids somewhere, then we will hit conflicting attributes and an assertion error will be thrown by `QueryPlan#transformUpWithNewOutput`. This PR takes a different approach. We canonicalize the plan first and then remove the alias-only project. Then we don't need `transformUpWithNewOutput` anymore as all attribute ids are normalized in the canonicalized plan. ### Why are the changes needed? fix potential bugs ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing tests Closes #42408 from cloud-fan/collect-metrics. Authored-by: Wenchen Fan Signed-off-by: Kent Yao --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fee5660017c7d..0b953fc2b61f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1080,13 +1080,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB def check(plan: LogicalPlan): Unit = plan.foreach { node => node match { case metrics @ CollectMetrics(name, _, _) => - val simplifiedMetrics = simplifyPlanForCollectedMetrics(metrics) + val simplifiedMetrics = simplifyPlanForCollectedMetrics(metrics.canonicalized) metricsMap.get(name) match { case Some(other) => - val simplifiedOther = simplifyPlanForCollectedMetrics(other) + val simplifiedOther = simplifyPlanForCollectedMetrics(other.canonicalized) // Exact duplicates are allowed. They can be the result // of a CTE that is used multiple times or a self join. - if (!simplifiedMetrics.sameResult(simplifiedOther)) { + if (simplifiedMetrics != simplifiedOther) { failAnalysis( errorClass = "DUPLICATED_METRICS_NAME", messageParameters = Map("metricName" -> name)) @@ -1111,7 +1111,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB * duplicates metric definition. */ private def simplifyPlanForCollectedMetrics(plan: LogicalPlan): LogicalPlan = { - plan.transformUpWithNewOutput { + plan.resolveOperators { case p: Project if p.projectList.size == p.child.output.size => val assignExprIdOnly = p.projectList.zip(p.child.output).forall { case (left: Alias, right: Attribute) => @@ -1119,9 +1119,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case _ => false } if (assignExprIdOnly) { - (p.child, p.output.zip(p.child.output)) + p.child } else { - (p, Nil) + p } } } From b19a83e236b2732c2af6d0a4d79ebcbca4105c59 Mon Sep 17 00:00:00 2001 From: itholic Date: Wed, 9 Aug 2023 20:32:48 +0900 Subject: [PATCH 02/12] [SPARK-43660][CONNECT][PS][FOLLOWUP] Remove JVM dependency for resample ### What changes were proposed in this pull request? This is follow-up for https://github.com/apache/spark/pull/41877 to remove JVM dependency. ### Why are the changes needed? To remove JVM dependency from Pandas API on Spark with Spark Connect. ### Does this PR introduce _any_ user-facing change? No, it's internal handling. ### How was this patch tested? The existing UT. Closes #42410 from itholic/resample_followup. Authored-by: itholic Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/resample.py | 26 +++++++++----------------- 1 file changed, 9 insertions(+), 17 deletions(-) diff --git a/python/pyspark/pandas/resample.py b/python/pyspark/pandas/resample.py index 30f8c9d31695e..0d2c3cc753cd1 100644 --- a/python/pyspark/pandas/resample.py +++ b/python/pyspark/pandas/resample.py @@ -67,7 +67,6 @@ scol_for, verify_temp_column_name, ) -from pyspark.sql.utils import is_remote from pyspark.pandas.spark.functions import timestampdiff @@ -145,22 +144,15 @@ def _agg_columns_scols(self) -> List[Column]: def get_make_interval( # type: ignore[return] self, unit: str, col: Union[Column, int, float] ) -> Column: - if is_remote(): - from pyspark.sql.connect.functions import lit, make_interval - - col = col if not isinstance(col, (int, float)) else lit(col) # type: ignore[assignment] - if unit == "MONTH": - return make_interval(months=col) # type: ignore - if unit == "HOUR": - return make_interval(hours=col) # type: ignore - if unit == "MINUTE": - return make_interval(mins=col) # type: ignore - if unit == "SECOND": - return make_interval(secs=col) # type: ignore - else: - sql_utils = SparkContext._active_spark_context._jvm.PythonSQLUtils - col = col._jc if isinstance(col, Column) else F.lit(col)._jc - return sql_utils.makeInterval(unit, col) + col = col if not isinstance(col, (int, float)) else F.lit(col) # type: ignore[assignment] + if unit == "MONTH": + return F.make_interval(months=col) # type: ignore + if unit == "HOUR": + return F.make_interval(hours=col) # type: ignore + if unit == "MINUTE": + return F.make_interval(mins=col) # type: ignore + if unit == "SECOND": + return F.make_interval(secs=col) # type: ignore def _bin_timestamp(self, origin: pd.Timestamp, ts_scol: Column) -> Column: key_type = self._resamplekey_type From e5e0f389dad678e664d5e9897d40a8765a0dc2e5 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 9 Aug 2023 20:33:50 +0900 Subject: [PATCH 03/12] Revert "[SPARK-43660][CONNECT][PS][FOLLOWUP] Remove JVM dependency for resample" This reverts commit b19a83e236b2732c2af6d0a4d79ebcbca4105c59. --- python/pyspark/pandas/resample.py | 26 +++++++++++++++++--------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/python/pyspark/pandas/resample.py b/python/pyspark/pandas/resample.py index 0d2c3cc753cd1..30f8c9d31695e 100644 --- a/python/pyspark/pandas/resample.py +++ b/python/pyspark/pandas/resample.py @@ -67,6 +67,7 @@ scol_for, verify_temp_column_name, ) +from pyspark.sql.utils import is_remote from pyspark.pandas.spark.functions import timestampdiff @@ -144,15 +145,22 @@ def _agg_columns_scols(self) -> List[Column]: def get_make_interval( # type: ignore[return] self, unit: str, col: Union[Column, int, float] ) -> Column: - col = col if not isinstance(col, (int, float)) else F.lit(col) # type: ignore[assignment] - if unit == "MONTH": - return F.make_interval(months=col) # type: ignore - if unit == "HOUR": - return F.make_interval(hours=col) # type: ignore - if unit == "MINUTE": - return F.make_interval(mins=col) # type: ignore - if unit == "SECOND": - return F.make_interval(secs=col) # type: ignore + if is_remote(): + from pyspark.sql.connect.functions import lit, make_interval + + col = col if not isinstance(col, (int, float)) else lit(col) # type: ignore[assignment] + if unit == "MONTH": + return make_interval(months=col) # type: ignore + if unit == "HOUR": + return make_interval(hours=col) # type: ignore + if unit == "MINUTE": + return make_interval(mins=col) # type: ignore + if unit == "SECOND": + return make_interval(secs=col) # type: ignore + else: + sql_utils = SparkContext._active_spark_context._jvm.PythonSQLUtils + col = col._jc if isinstance(col, Column) else F.lit(col)._jc + return sql_utils.makeInterval(unit, col) def _bin_timestamp(self, origin: pd.Timestamp, ts_scol: Column) -> Column: key_type = self._resamplekey_type From 0b757d3b610a90e70828d5e21521810f923c6aed Mon Sep 17 00:00:00 2001 From: zhyhimont Date: Wed, 9 Aug 2023 20:35:15 +0900 Subject: [PATCH 04/12] [SPARK-42620][PS] Add `inclusive` parameter for (DataFrame|Series).between_time ### What changes were proposed in this pull request? Add `inclusive` parameter for (DataFrame|Series).between_time to support the pandas 2.0.0 ### Why are the changes needed? When pandas 2.0.0 is released, we should match the behavior in pandas API on Spark. ### Does this PR introduce _any_ user-facing change? yes, the API changes Before: ` (DataFrame|Series).between_time(start_time, end_time, include_start, include_end, axis)` After: ` (DataFrame|Series).between_time(start_time, end_time, inclusive, axis)` ### How was this patch tested? Unit tests were updated Closes #40370 from dzhigimont/SPARK-42620-ZH. Lead-authored-by: zhyhimont Co-authored-by: Zhyhimont Dmitry Co-authored-by: Zhyhimont Dmitry Signed-off-by: Hyukjin Kwon --- .../migration_guide/pyspark_upgrade.rst | 2 ++ python/pyspark/errors/error_classes.py | 5 +++ python/pyspark/pandas/frame.py | 30 +++++++++-------- python/pyspark/pandas/series.py | 18 +++------- .../pandas/tests/frame/test_reindexing.py | 33 ++++++++++++++++--- .../pandas/tests/series/test_compute.py | 32 +++++++++++++++--- 6 files changed, 86 insertions(+), 34 deletions(-) diff --git a/python/docs/source/migration_guide/pyspark_upgrade.rst b/python/docs/source/migration_guide/pyspark_upgrade.rst index 1b247d4622787..da49719579ad2 100644 --- a/python/docs/source/migration_guide/pyspark_upgrade.rst +++ b/python/docs/source/migration_guide/pyspark_upgrade.rst @@ -32,6 +32,8 @@ Upgrading from PySpark 3.5 to 4.0 * In Spark 4.0, ``na_sentinel`` parameter from ``Index.factorize`` and `Series.factorize`` has been removed from pandas API on Spark, use ``use_na_sentinel`` instead. * In Spark 4.0, ``inplace`` parameter from ``Categorical.add_categories``, ``Categorical.remove_categories``, ``Categorical.set_categories``, ``Categorical.rename_categories``, ``Categorical.reorder_categories``, ``Categorical.as_ordered``, ``Categorical.as_unordered`` have been removed from pandas API on Spark. * In Spark 4.0, ``closed`` parameter from ``ps.date_range`` has been removed from pandas API on Spark. +* In Spark 4.0, ``include_start`` and ``include_end`` parameters from ``DataFrame.between_time`` have been removed from pandas API on Spark, use ``inclusive`` instead. +* In Spark 4.0, ``include_start`` and ``include_end`` parameters from ``Series.between_time`` have been removed from pandas API on Spark, use ``inclusive`` instead. Upgrading from PySpark 3.3 to 3.4 diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index bc32afeb87a9f..c803347836295 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -853,6 +853,11 @@ "Value `` cannot be accessed inside tasks." ] }, + "VALUE_NOT_ALLOWED" : { + "message" : [ + "Value for `` has to be amongst the following values: ." + ] + }, "VALUE_NOT_ANY_OR_ALL" : { "message" : [ "Value for `` must be 'any' or 'all', got ''." diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 65c43eb7cf42c..8fbe1b8f926af 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -59,6 +59,8 @@ ) from pandas.tseries.frequencies import DateOffset, to_offset +from pyspark.errors import PySparkValueError + if TYPE_CHECKING: from pandas.io.formats.style import Styler @@ -3501,14 +3503,11 @@ class locomotion ).resolved_copy return DataFrame(internal) - # TODO(SPARK-42620): Add `inclusive` parameter and replace `include_start` & `include_end`. - # See https://github.com/pandas-dev/pandas/issues/43248 def between_time( self, start_time: Union[datetime.time, str], end_time: Union[datetime.time, str], - include_start: bool = True, - include_end: bool = True, + inclusive: str = "both", axis: Axis = 0, ) -> "DataFrame": """ @@ -3523,15 +3522,10 @@ def between_time( Initial time as a time filter limit. end_time : datetime.time or str End time as a time filter limit. - include_start : bool, default True - Whether the start time needs to be included in the result. - - .. deprecated:: 3.4.0 + inclusive : {"both", "neither", "left", "right"}, default "both" + Include boundaries; whether to set each bound as closed or open. - include_end : bool, default True - Whether the end time needs to be included in the result. - - .. deprecated:: 3.4.0 + .. versionadded:: 4.0.0 axis : {0 or 'index', 1 or 'columns'}, default 0 Determine range time on index or columns value. @@ -3586,6 +3580,16 @@ def between_time( if not isinstance(self.index, ps.DatetimeIndex): raise TypeError("Index must be DatetimeIndex") + allowed_inclusive_values = ["left", "right", "both", "neither"] + if inclusive not in allowed_inclusive_values: + raise PySparkValueError( + error_class="VALUE_NOT_ALLOWED", + message_parameters={ + "arg_name": "inclusive", + "allowed_values": str(allowed_inclusive_values), + }, + ) + psdf = self.copy() psdf.index.name = verify_temp_column_name(psdf, "__index_name__") return_types = [psdf.index.dtype] + list(psdf.dtypes) @@ -3593,7 +3597,7 @@ def between_time( def pandas_between_time( # type: ignore[no-untyped-def] pdf, ) -> ps.DataFrame[return_types]: # type: ignore[valid-type] - return pdf.between_time(start_time, end_time, include_start, include_end).reset_index() + return pdf.between_time(start_time, end_time, inclusive).reset_index() # apply_batch will remove the index of the pandas-on-Spark DataFrame and attach a # default index, which will never be used. Use "distributed" index as a dummy to diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py index a74f36986f3b5..d1b1d7631795c 100644 --- a/python/pyspark/pandas/series.py +++ b/python/pyspark/pandas/series.py @@ -6718,14 +6718,11 @@ def align( return (left_ser.copy(), right.copy()) if copy else (left_ser, right) - # TODO(SPARK-42620): Add `inclusive` parameter and replace `include_start` & `include_end`. - # See https://github.com/pandas-dev/pandas/issues/43248 def between_time( self, start_time: Union[datetime.time, str], end_time: Union[datetime.time, str], - include_start: bool = True, - include_end: bool = True, + inclusive: str = "both", axis: Axis = 0, ) -> "Series": """ @@ -6740,15 +6737,10 @@ def between_time( Initial time as a time filter limit. end_time : datetime.time or str End time as a time filter limit. - include_start : bool, default True - Whether the start time needs to be included in the result. + inclusive : {"both", "neither", "left", "right"}, default "both" + Include boundaries; whether to set each bound as closed or open. - .. deprecated:: 3.4.0 - - include_end : bool, default True - Whether the end time needs to be included in the result. - - .. deprecated:: 3.4.0 + .. versionadded:: 4.0.0 axis : {0 or 'index', 1 or 'columns'}, default 0 Determine range time on index or columns value. @@ -6787,7 +6779,7 @@ def between_time( dtype: int64 """ return first_series( - self.to_frame().between_time(start_time, end_time, include_start, include_end, axis) + self.to_frame().between_time(start_time, end_time, inclusive, axis) ).rename(self.name) def at_time( diff --git a/python/pyspark/pandas/tests/frame/test_reindexing.py b/python/pyspark/pandas/tests/frame/test_reindexing.py index ea9a75b2d799e..f9d39aec1add7 100644 --- a/python/pyspark/pandas/tests/frame/test_reindexing.py +++ b/python/pyspark/pandas/tests/frame/test_reindexing.py @@ -22,6 +22,7 @@ from pandas.tseries.offsets import DateOffset from pyspark import pandas as ps +from pyspark.errors import PySparkValueError from pyspark.pandas.config import option_context from pyspark.testing.pandasutils import ComparisonTestBase from pyspark.testing.sqlutils import SQLTestUtils @@ -115,10 +116,6 @@ def test_at_time(self): with self.assertRaisesRegex(TypeError, "Index must be DatetimeIndex"): psdf.at_time("0:15") - @unittest.skipIf( - LooseVersion(pd.__version__) >= LooseVersion("2.0.0"), - "TODO(SPARK-43557): Enable DataFrameSlowTests.test_between_time for pandas 2.0.0.", - ) def test_between_time(self): idx = pd.date_range("2018-04-09", periods=4, freq="1D20min") pdf = pd.DataFrame({"A": [1, 2, 3, 4]}, index=idx) @@ -168,6 +165,34 @@ def test_between_time(self): with self.assertRaisesRegex(TypeError, "Index must be DatetimeIndex"): psdf.between_time("0:15", "0:45") + psdf = ps.from_pandas(pdf) + self.assert_eq( + pdf.between_time("0:15", "0:45", inclusive="neither").sort_index(), + psdf.between_time("0:15", "0:45", inclusive="neither").sort_index(), + ) + + self.assert_eq( + pdf.between_time("0:15", "0:45", inclusive="left").sort_index(), + psdf.between_time("0:15", "0:45", inclusive="left").sort_index(), + ) + + self.assert_eq( + pdf.between_time("0:15", "0:45", inclusive="right").sort_index(), + psdf.between_time("0:15", "0:45", inclusive="right").sort_index(), + ) + + with self.assertRaises(PySparkValueError) as ctx: + psdf.between_time("0:15", "0:45", inclusive="") + + self.check_error( + exception=ctx.exception, + error_class="VALUE_NOT_ALLOWED", + message_parameters={ + "arg_name": "inclusive", + "allowed_values": str(["left", "right", "both", "neither"]), + }, + ) + def test_drop(self): pdf = pd.DataFrame({"x": [1, 2], "y": [3, 4], "z": [5, 6]}, index=np.random.rand(2)) psdf = ps.from_pandas(pdf) diff --git a/python/pyspark/pandas/tests/series/test_compute.py b/python/pyspark/pandas/tests/series/test_compute.py index 784bf29e1a25b..4efa9a784ebfd 100644 --- a/python/pyspark/pandas/tests/series/test_compute.py +++ b/python/pyspark/pandas/tests/series/test_compute.py @@ -22,6 +22,7 @@ import pandas as pd from pyspark import pandas as ps +from pyspark.errors import PySparkValueError from pyspark.testing.pandasutils import ComparisonTestBase from pyspark.testing.sqlutils import SQLTestUtils @@ -555,10 +556,6 @@ def test_between(self): with self.assertWarns(FutureWarning): psser.between(1, 4, inclusive=True) - @unittest.skipIf( - LooseVersion(pd.__version__) >= LooseVersion("2.0.0"), - "TODO(SPARK-43479): Enable SeriesTests.test_between_time for pandas 2.0.0.", - ) def test_between_time(self): idx = pd.date_range("2018-04-09", periods=4, freq="1D20min") pser = pd.Series([1, 2, 3, 4], index=idx) @@ -582,6 +579,33 @@ def test_between_time(self): psser.between_time("0:15", "0:45").sort_index(), ) + self.assert_eq( + pser.between_time("0:15", "0:45", inclusive="neither").sort_index(), + psser.between_time("0:15", "0:45", inclusive="neither").sort_index(), + ) + + self.assert_eq( + pser.between_time("0:15", "0:45", inclusive="left").sort_index(), + psser.between_time("0:15", "0:45", inclusive="left").sort_index(), + ) + + self.assert_eq( + pser.between_time("0:15", "0:45", inclusive="right").sort_index(), + psser.between_time("0:15", "0:45", inclusive="right").sort_index(), + ) + + with self.assertRaises(PySparkValueError) as ctx: + psser.between_time("0:15", "0:45", inclusive="") + + self.check_error( + exception=ctx.exception, + error_class="VALUE_NOT_ALLOWED", + message_parameters={ + "arg_name": "inclusive", + "allowed_values": str(["left", "right", "both", "neither"]), + }, + ) + def test_at_time(self): idx = pd.date_range("2018-04-09", periods=4, freq="1D20min") pser = pd.Series([1, 2, 3, 4], index=idx) From 27c5a1f9f0e322fad0da300afdb75eadd8224b15 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 9 Aug 2023 20:42:20 +0900 Subject: [PATCH 05/12] [SPARK-43429][CONNECT] Deflake SparkSessionSuite ### What changes were proposed in this pull request? This PR tries to fix flakiness in the `SparkSessionSuite.active session in multiple threads` test. There was a chance that modification could happen before the other thread could check the state. This PR decouples modifcations from checks. ### Why are the changes needed? Flaky tests are no bueno. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? It is a test. Closes #42406 from hvanhovell/SPARK-43429-deflake. Authored-by: Herman van Hovell Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/SparkSessionSuite.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala index f06744399f833..2d7ded2d68858 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala @@ -171,42 +171,74 @@ class SparkSessionSuite extends ConnectFunSuite { try { val script1 = execute { phaser => + // Step 0 - check initial state phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.contains(session1)) assert(SparkSession.getActiveSession.contains(session2)) + // Step 1 - new active session in script 2 + phaser.arriveAndAwaitAdvance() + + // Step2 - script 1 is unchanged, script 2 has new active session phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.contains(session1)) assert(SparkSession.getActiveSession.contains(session2)) + + // Step 3 - close session 1, no more default session in both scripts + phaser.arriveAndAwaitAdvance() session1.close() + // Step 4 - no default session, same active session. phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.isEmpty) assert(SparkSession.getActiveSession.contains(session2)) + + // Step 5 - clear active session in script 1 + phaser.arriveAndAwaitAdvance() SparkSession.clearActiveSession() + // Step 6 - no default/no active session in script 1, script2 unchanged. phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.isEmpty) assert(SparkSession.getActiveSession.isEmpty) + + // Step 7 - close active session in script2 + phaser.arriveAndAwaitAdvance() } val script2 = execute { phaser => + // Step 0 - check initial state phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.contains(session1)) assert(SparkSession.getActiveSession.contains(session2)) + + // Step 1 - new active session in script 2 + phaser.arriveAndAwaitAdvance() SparkSession.clearActiveSession() val internalSession = SparkSession.builder().remote(connectionString3).getOrCreate() + // Step2 - script 1 is unchanged, script 2 has new active session phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.contains(session1)) assert(SparkSession.getActiveSession.contains(internalSession)) + // Step 3 - close session 1, no more default session in both scripts + phaser.arriveAndAwaitAdvance() + + // Step 4 - no default session, same active session. phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.isEmpty) assert(SparkSession.getActiveSession.contains(internalSession)) + // Step 5 - clear active session in script 1 + phaser.arriveAndAwaitAdvance() + + // Step 6 - no default/no active session in script 1, script2 unchanged. phaser.arriveAndAwaitAdvance() assert(SparkSession.getDefaultSession.isEmpty) assert(SparkSession.getActiveSession.contains(internalSession)) + + // Step 7 - close active session in script2 + phaser.arriveAndAwaitAdvance() internalSession.close() assert(SparkSession.getActiveSession.isEmpty) } From be9ffb37585fe421705ceaa52fe49b89c50703a3 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 9 Aug 2023 15:58:18 +0200 Subject: [PATCH 06/12] [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder ### What changes were proposed in this pull request? Make the Spark Connect Dataset use Encoder instead of AgnosticEncoder ### Why are the changes needed? We want to improve binary compatibility between the Spark Connect Scala Client and the original sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It changes the type of `Dataset.encoder` from `AgnosticEncoder` to `Encoder`. ### How was this patch tested? Existing tests. Closes #42396 from hvanhovell/SPARK-44720. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/Dataset.scala | 87 ++++++++++--------- .../spark/sql/KeyValueGroupedDataset.scala | 6 +- .../sql/streaming/DataStreamWriter.scala | 2 +- .../CheckConnectJvmClientCompatibility.scala | 3 - 4 files changed, 50 insertions(+), 48 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 5f263903c8bbc..2d72ea6bda8f3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -128,11 +128,13 @@ import org.apache.spark.util.SparkClassUtils class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, - val encoder: AgnosticEncoder[T]) + val encoder: Encoder[T]) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) + private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) + override def toString: String = { try { val builder = new mutable.StringBuilder @@ -828,7 +830,7 @@ class Dataset[T] private[sql] ( } private def buildSort(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -878,8 +880,8 @@ class Dataset[T] private[sql] ( ProductEncoder[(T, U)]( ClassTag(SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple2")), Seq( - EncoderField(s"_1", this.encoder, leftNullable, Metadata.empty), - EncoderField(s"_2", other.encoder, rightNullable, Metadata.empty))) + EncoderField(s"_1", this.agnosticEncoder, leftNullable, Metadata.empty), + EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty))) sparkSession.newDataset(tupleEncoder) { builder => val joinBuilder = builder.getJoinBuilder @@ -889,8 +891,8 @@ class Dataset[T] private[sql] ( .setJoinType(joinTypeValue) .setJoinCondition(condition.expr) .setJoinDataType(joinBuilder.getJoinDataTypeBuilder - .setIsLeftStruct(this.encoder.isStruct) - .setIsRightStruct(other.encoder.isStruct)) + .setIsLeftStruct(this.agnosticEncoder.isStruct) + .setIsRightStruct(other.agnosticEncoder.isStruct)) } } @@ -1010,13 +1012,13 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ @scala.annotation.varargs - def hint(name: String, parameters: Any*): Dataset[T] = sparkSession.newDataset(encoder) { - builder => + def hint(name: String, parameters: Any*): Dataset[T] = + sparkSession.newDataset(agnosticEncoder) { builder => builder.getHintBuilder .setInput(plan.getRoot) .setName(name) .addAllParameters(parameters.map(p => functions.lit(p).expr).asJava) - } + } private def getPlanId: Option[Long] = if (plan.getRoot.hasCommon && plan.getRoot.getCommon.hasPlanId) { @@ -1056,7 +1058,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def as(alias: String): Dataset[T] = sparkSession.newDataset(encoder) { builder => + def as(alias: String): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getSubqueryAliasBuilder .setInput(plan.getRoot) .setAlias(alias) @@ -1238,8 +1240,9 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def filter(condition: Column): Dataset[T] = sparkSession.newDataset(encoder) { builder => - builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) + def filter(condition: Column): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => + builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) } /** @@ -1355,12 +1358,12 @@ class Dataset[T] private[sql] ( def reduce(func: (T, T) => T): T = { val udf = ScalarUserDefinedFunction( function = func, - inputEncoders = encoder :: encoder :: Nil, - outputEncoder = encoder) + inputEncoders = agnosticEncoder :: agnosticEncoder :: Nil, + outputEncoder = agnosticEncoder) val reduceExpr = Column.fn("reduce", udf.apply(col("*"), col("*"))).expr val result = sparkSession - .newDataset(encoder) { builder => + .newDataset(agnosticEncoder) { builder => builder.getAggregateBuilder .setInput(plan.getRoot) .addAggregateExpressions(reduceExpr) @@ -1718,7 +1721,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def limit(n: Int): Dataset[T] = sparkSession.newDataset(encoder) { builder => + def limit(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getLimitBuilder .setInput(plan.getRoot) .setLimit(n) @@ -1730,7 +1733,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def offset(n: Int): Dataset[T] = sparkSession.newDataset(encoder) { builder => + def offset(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getOffsetBuilder .setInput(plan.getRoot) .setOffset(n) @@ -1739,7 +1742,7 @@ class Dataset[T] private[sql] ( private def buildSetOp(right: Dataset[T], setOpType: proto.SetOperation.SetOpType)( f: proto.SetOperation.Builder => Unit): Dataset[T] = { checkSameSparkSession(right) - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => f( builder.getSetOpBuilder .setSetOpType(setOpType) @@ -2012,7 +2015,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSampleBuilder .setInput(plan.getRoot) .setWithReplacement(withReplacement) @@ -2080,7 +2083,7 @@ class Dataset[T] private[sql] ( normalizedCumWeights .sliding(2) .map { case Array(low, high) => - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSampleBuilder .setInput(sortedInput) .setWithReplacement(false) @@ -2401,15 +2404,16 @@ class Dataset[T] private[sql] ( private def buildDropDuplicates( columns: Option[Seq[String]], - withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(encoder) { builder => - val dropBuilder = builder.getDeduplicateBuilder - .setInput(plan.getRoot) - .setWithinWatermark(withinWaterMark) - if (columns.isDefined) { - dropBuilder.addAllColumnNames(columns.get.asJava) - } else { - dropBuilder.setAllColumnsAsKeys(true) - } + withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => + val dropBuilder = builder.getDeduplicateBuilder + .setInput(plan.getRoot) + .setWithinWatermark(withinWaterMark) + if (columns.isDefined) { + dropBuilder.addAllColumnNames(columns.get.asJava) + } else { + dropBuilder.setAllColumnsAsKeys(true) + } } /** @@ -2630,9 +2634,9 @@ class Dataset[T] private[sql] ( def filter(func: T => Boolean): Dataset[T] = { val udf = ScalarUserDefinedFunction( function = func, - inputEncoders = encoder :: Nil, + inputEncoders = agnosticEncoder :: Nil, outputEncoder = PrimitiveBooleanEncoder) - sparkSession.newDataset[T](encoder) { builder => + sparkSession.newDataset[T](agnosticEncoder) { builder => builder.getFilterBuilder .setInput(plan.getRoot) .setCondition(udf.apply(col("*")).expr) @@ -2683,7 +2687,7 @@ class Dataset[T] private[sql] ( val outputEncoder = encoderFor[U] val udf = ScalarUserDefinedFunction( function = func, - inputEncoders = encoder :: Nil, + inputEncoders = agnosticEncoder :: Nil, outputEncoder = outputEncoder) sparkSession.newDataset(outputEncoder) { builder => builder.getMapPartitionsBuilder @@ -2785,7 +2789,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def tail(n: Int): Array[T] = { - val lastN = sparkSession.newDataset(encoder) { builder => + val lastN = sparkSession.newDataset(agnosticEncoder) { builder => builder.getTailBuilder .setInput(plan.getRoot) .setLimit(n) @@ -2856,7 +2860,7 @@ class Dataset[T] private[sql] ( } private def buildRepartition(numPartitions: Int, shuffle: Boolean): Dataset[T] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getRepartitionBuilder .setInput(plan.getRoot) .setNumPartitions(numPartitions) @@ -2866,11 +2870,12 @@ class Dataset[T] private[sql] ( private def buildRepartitionByExpression( numPartitions: Option[Int], - partitionExprs: Seq[Column]): Dataset[T] = sparkSession.newDataset(encoder) { builder => - val repartitionBuilder = builder.getRepartitionByExpressionBuilder - .setInput(plan.getRoot) - .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) - numPartitions.foreach(repartitionBuilder.setNumPartitions) + partitionExprs: Seq[Column]): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => + val repartitionBuilder = builder.getRepartitionByExpressionBuilder + .setInput(plan.getRoot) + .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) + numPartitions.foreach(repartitionBuilder.setNumPartitions) } /** @@ -3183,7 +3188,7 @@ class Dataset[T] private[sql] ( * @since 3.5.0 */ def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getWithWatermarkBuilder .setInput(plan.getRoot) .setEventTime(eventTime) @@ -3251,7 +3256,7 @@ class Dataset[T] private[sql] ( sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA) } - def collectResult(): SparkResult[T] = sparkSession.execute(plan, encoder) + def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder) private[sql] def withResult[E](f: SparkResult[T] => E): E = { val result = collectResult() diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index e67ef1c0fa7e2..202891c66d748 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -988,15 +988,15 @@ private object KeyValueGroupedDatasetImpl { groupingFunc: V => K): KeyValueGroupedDatasetImpl[K, V, K, V] = { val gf = ScalarUserDefinedFunction( function = groupingFunc, - inputEncoders = ds.encoder :: Nil, // Using the original value and key encoders + inputEncoders = ds.agnosticEncoder :: Nil, // Using the original value and key encoders outputEncoder = kEncoder) new KeyValueGroupedDatasetImpl( ds.sparkSession, ds.plan, kEncoder, kEncoder, - ds.encoder, - ds.encoder, + ds.agnosticEncoder, + ds.agnosticEncoder, Arrays.asList(gf.apply(col("*")).expr), UdfUtils.identical(), () => ds.map(groupingFunc)(kEncoder)) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b395a2d073d6d..b9aa1f5bc5838 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -216,7 +216,7 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { * @since 3.5.0 */ def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { - val serialized = SparkSerDeUtils.serialize(ForeachWriterPacket(writer, ds.encoder)) + val serialized = SparkSerDeUtils.serialize(ForeachWriterPacket(writer, ds.agnosticEncoder)) val scalaWriterBuilder = proto.ScalarScalaUDF .newBuilder() .setPayload(ByteString.copyFrom(serialized)) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index d380a1bbb653e..4439a5f3e2adb 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -181,7 +181,6 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener$"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.queryExecution"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.encoder"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.sqlContext"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.metadataColumn"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.selectUntyped"), // protected @@ -334,8 +333,6 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.Dataset.plan" ), // developer API - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.sql.Dataset.encoder"), ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.Dataset.collectResult"), From 87298db43d9a33fa3a3986f274442a17aad74dc3 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Wed, 9 Aug 2023 10:27:07 -0700 Subject: [PATCH 07/12] [SPARK-44503][SQL] Project any PARTITION BY expressions not already returned from Python UDTF TABLE arguments ### What changes were proposed in this pull request? This PR adds a projection when any Python UDTF TABLE argument contains PARTITION BY expressions that are not simple attributes that are already present in the output of the relation. For example: ``` CREATE TABLE t(d DATE, y INT) USING PARQUET; INSERT INTO t VALUES ... SELECT * FROM UDTF(TABLE(t) PARTITION BY EXTRACT(YEAR FROM d) ORDER BY y ASC); ``` This will generate a plan like: ``` +- Sort (y ASC) +- RepartitionByExpressions (partition_by_0) +- Project (t.d, t.y, EXTRACT(YEAR FROM t.d) AS partition_by_0) +- LogicalRelation "t" ``` ### Why are the changes needed? We project the PARTITION BY expressions so that their resulting values appear in attributes that the Python UDTF interpreter can simply inspect in order to know when the partition boundaries have changed. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR adds unit test coverage. Closes #42351 from dtenedor/partition-by-execution. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- ...ctionTableSubqueryArgumentExpression.scala | 77 +++++++++-- .../execution/python/PythonUDTFSuite.scala | 127 ++++++++++++++++-- 2 files changed, 184 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index e7a4888125df1..daa0751eedf22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -104,23 +104,80 @@ case class FunctionTableSubqueryArgumentExpression( // the query plan. var subquery = plan if (partitionByExpressions.nonEmpty) { - subquery = RepartitionByExpression( - partitionExpressions = partitionByExpressions, - child = subquery, - optNumPartitions = None) + // Add a projection to project each of the partitioning expressions that it is not a simple + // attribute that is already present in the plan output. Then add a sort operation by the + // partition keys (plus any explicit ORDER BY items) since after the hash-based shuffle + // operation, the rows from several partitions may arrive interleaved. In this way, the Python + // UDTF evaluator is able to inspect the values of the partitioning expressions for adjacent + // rows in order to determine when each partition ends and the next one begins. + subquery = Project( + projectList = subquery.output ++ extraProjectedPartitioningExpressions, + child = subquery) + val partitioningAttributes = partitioningExpressionIndexes.map(i => subquery.output(i)) + subquery = Sort( + order = partitioningAttributes.map(e => SortOrder(e, Ascending)) ++ orderByExpressions, + global = false, + child = RepartitionByExpression( + partitionExpressions = partitioningAttributes, + optNumPartitions = None, + child = subquery)) } if (withSinglePartition) { subquery = Repartition( numPartitions = 1, shuffle = true, child = subquery) - } - if (orderByExpressions.nonEmpty) { - subquery = Sort( - order = orderByExpressions, - global = false, - child = subquery) + if (orderByExpressions.nonEmpty) { + subquery = Sort( + order = orderByExpressions, + global = false, + child = subquery) + } } Project(Seq(Alias(CreateStruct(subquery.output), "c")()), subquery) } + + /** + * These are the indexes of the PARTITION BY expressions within the concatenation of the child's + * output attributes and the [[extraProjectedPartitioningExpressions]]. We send these indexes to + * the Python UDTF evaluator so it knows which expressions to compare on adjacent rows to know + * when the partition has changed. + */ + lazy val partitioningExpressionIndexes: Seq[Int] = { + val extraPartitionByExpressionsToIndexes: Map[Expression, Int] = + extraProjectedPartitioningExpressions.map(_.child).zipWithIndex.toMap + partitionByExpressions.map { e => + subqueryOutputs.get(e).getOrElse { + extraPartitionByExpressionsToIndexes.get(e).get + plan.output.length + } + } + } + + private lazy val extraProjectedPartitioningExpressions: Seq[Alias] = { + partitionByExpressions.filter { e => + !subqueryOutputs.contains(e) + }.zipWithIndex.map { case (expr, index) => + Alias(expr, s"partition_by_$index")() + } + } + + private lazy val subqueryOutputs: Map[Expression, Int] = plan.output.zipWithIndex.toMap } + +object FunctionTableSubqueryArgumentExpression { + /** + * Returns a sequence of zero-based integer indexes identifying the values of a Python UDTF's + * 'eval' method's *args list that correspond to partitioning columns of the input TABLE argument. + */ + def partitionChildIndexes(udtfArguments: Seq[Expression]): Seq[Int] = { + udtfArguments.zipWithIndex.flatMap { case (expr, index) => + expr match { + case f: FunctionTableSubqueryArgumentExpression => + f.partitioningExpressionIndexes.map(_ + index) + case _ => + Seq() + } + } + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala index 8f1bf172bbdac..43f61a7c61e8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution.python import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest, Row} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Repartition, RepartitionByExpression, Sort, SubqueryAlias} +import org.apache.spark.sql.catalyst.expressions.{Add, Alias, FunctionTableSubqueryArgumentExpression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, OneRowRelation, Project, Repartition, RepartitionByExpression, Sort, SubqueryAlias} import org.apache.spark.sql.functions.lit import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -112,7 +113,9 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { test("SPARK-44503: Specify PARTITION BY and ORDER BY for TABLE arguments") { // Positive tests assume(shouldTestPythonUDFs) - def failure(plan: LogicalPlan): Unit = fail(s"Unexpected plan: $plan") + def failure(plan: LogicalPlan): Unit = { + fail(s"Unexpected plan: $plan") + } sql( """ |SELECT * FROM testUDTF( @@ -120,8 +123,12 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { | PARTITION BY X) |""".stripMargin).queryExecution.analyzed .collectFirst { case r: RepartitionByExpression => r }.get match { - case RepartitionByExpression(_, SubqueryAlias(_, _: LocalRelation), _, _) => - case other => failure(other) + case RepartitionByExpression( + _, Project( + _, SubqueryAlias( + _, _: LocalRelation)), _, _) => + case other => + failure(other) } sql( """ @@ -130,8 +137,11 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { | WITH SINGLE PARTITION) |""".stripMargin).queryExecution.analyzed .collectFirst { case r: Repartition => r }.get match { - case Repartition(1, true, SubqueryAlias(_, _: LocalRelation)) => - case other => failure(other) + case Repartition( + 1, true, SubqueryAlias( + _, _: LocalRelation)) => + case other => + failure(other) } sql( """ @@ -140,8 +150,13 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { | PARTITION BY SUBSTR(X, 2) ORDER BY (X, Y)) |""".stripMargin).queryExecution.analyzed .collectFirst { case r: Sort => r }.get match { - case Sort(_, false, RepartitionByExpression(_, SubqueryAlias(_, _: LocalRelation), _, _)) => - case other => failure(other) + case Sort( + _, false, RepartitionByExpression( + _, Project( + _, SubqueryAlias( + _, _: LocalRelation)), _, _)) => + case other => + failure(other) } sql( """ @@ -150,8 +165,12 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { | WITH SINGLE PARTITION ORDER BY (X, Y)) |""".stripMargin).queryExecution.analyzed .collectFirst { case r: Sort => r }.get match { - case Sort(_, false, Repartition(1, true, SubqueryAlias(_, _: LocalRelation))) => - case other => failure(other) + case Sort( + _, false, Repartition( + 1, true, SubqueryAlias( + _, _: LocalRelation))) => + case other => + failure(other) } // Negative tests withTable("t") { @@ -172,4 +191,92 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { stop = 30)) } } + + test("SPARK-44503: Compute partition child indexes for various UDTF argument lists") { + // Each of the following tests calls the PythonUDTF.partitionChildIndexes with a list of + // expressions and then checks the PARTITION BY child expression indexes that come out. + val projectList = Seq( + Alias(Literal(42), "a")(), + Alias(Literal(43), "b")()) + val projectTwoValues = Project( + projectList = projectList, + child = OneRowRelation()) + // There are no UDTF TABLE arguments, so there are no PARTITION BY child expression indexes. + val partitionChildIndexes = FunctionTableSubqueryArgumentExpression.partitionChildIndexes(_) + assert(partitionChildIndexes(Seq( + Literal(41))) == + Seq.empty[Int]) + assert(partitionChildIndexes(Seq( + Literal(41), + Literal("abc"))) == + Seq.empty[Int]) + // The UDTF TABLE argument has no PARTITION BY expressions, so there are no PARTITION BY child + // expression indexes. + assert(partitionChildIndexes(Seq( + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues))) == + Seq.empty[Int]) + // The UDTF TABLE argument has two PARTITION BY expressions which are equal to the output + // attributes from the provided relation, in order. Therefore the PARTITION BY child expression + // indexes are 0 and 1. + assert(partitionChildIndexes(Seq( + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = projectTwoValues.output))) == + Seq(0, 1)) + // The UDTF TABLE argument has one PARTITION BY expression which is equal to the first output + // attribute from the provided relation. Therefore the PARTITION BY child expression index is 0. + assert(partitionChildIndexes(Seq( + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = Seq(projectList.head.toAttribute)))) == + Seq(0)) + // The UDTF TABLE argument has one PARTITION BY expression which is equal to the second output + // attribute from the provided relation. Therefore the PARTITION BY child expression index is 1. + assert(partitionChildIndexes(Seq( + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = Seq(projectList.last.toAttribute)))) == + Seq(1)) + // The UDTF has one scalar argument, then one TABLE argument, then another scalar argument. The + // TABLE argument has two PARTITION BY expressions which are equal to the output attributes from + // the provided relation, in order. Therefore the PARTITION BY child expression indexes are 1 + // and 2, because they begin at an offset of 1 from the zero-based start of the list of values + // provided to the UDTF 'eval' method. + assert(partitionChildIndexes(Seq( + Literal(41), + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = projectTwoValues.output), + Literal("abc"))) == + Seq(1, 2)) + // Same as above, but the PARTITION BY expressions are new expressions which must be projected + // after all the attributes from the relation provided to the UDTF TABLE argument. Therefore the + // PARTITION BY child indexes are 3 and 4 because they begin at an offset of 3 from the + // zero-based start of the list of values provided to the UDTF 'eval' method. + assert(partitionChildIndexes(Seq( + Literal(41), + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = Seq(Literal(42), Literal(43))), + Literal("abc"))) == + Seq(3, 4)) + // Same as above, but the PARTITION BY list comprises just one addition expression. + assert(partitionChildIndexes(Seq( + Literal(41), + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = Seq(Add(projectList.head.toAttribute, Literal(1)))), + Literal("abc"))) == + Seq(3)) + // Same as above, but the PARTITION BY list comprises one literal value and one addition + // expression. + assert(partitionChildIndexes(Seq( + Literal(41), + FunctionTableSubqueryArgumentExpression( + plan = projectTwoValues, + partitionByExpressions = Seq(Literal(42), Add(projectList.head.toAttribute, Literal(1)))), + Literal("abc"))) == + Seq(3, 4)) + } } From 4db378fae30733cbd2be41e95a3cd8ad2184e06f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 9 Aug 2023 15:25:33 -0700 Subject: [PATCH 08/12] [SPARK-44745][DOCS][K8S] Document shuffle data recovery from the remounted K8s PVCs ### What changes were proposed in this pull request? This PR aims to document an example of shuffle data recovery configuration from the remounted K8s PVCs. ### Why are the changes needed? This will help the users use this feature more easily. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review because this is a doc-only change. ![Screenshot 2023-08-09 at 1 39 48 PM](https://github.com/apache/spark/assets/9700541/8cc7240b-570d-4c2e-b90a-54795c18df0a) ``` $ kubectl logs -f xxx-exec-16 | grep Kube ... 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Try to recover shuffle data. 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Found 192 files 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Try to recover /data/spark-x/executor-x/blockmgr-41a810ea-9503-447b-afc7-1cb104cd03cf/11/shuffle_0_11160_0.data 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Try to recover /data/spark-x/executor-x/blockmgr-41a810ea-9503-447b-afc7-1cb104cd03cf/0e/shuffle_0_10063_0.data 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Try to recover /data/spark-x/executor-x/blockmgr-41a810ea-9503-447b-afc7-1cb104cd03cf/0e/shuffle_0_10283_0.data 23/08/09 21:09:21 INFO KubernetesLocalDiskShuffleExecutorComponents: Ignore a non-shuffle block file. ``` Closes #42417 from dongjoon-hyun/SPARK-44745. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index d3953592c4ee8..707a76196f3ab 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -394,6 +394,13 @@ spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount. spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false ``` +To enable shuffle data recovery feature via the built-in `KubernetesLocalDiskShuffleDataIO` plugin, we need to have the followings. You may want to enable `spark.kubernetes.driver.waitToReusePersistentVolumeClaim` additionally. + +``` +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data/spark-x/executor-x +spark.shuffle.sort.io.plugin.class=org.apache.spark.shuffle.KubernetesLocalDiskShuffleDataIO +``` + If no volume is set as local storage, Spark uses temporary scratch space to spill data to disk during shuffles and other operations. When using Kubernetes as the resource manager the pods will be created with an [emptyDir](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir) volume mounted for each directory listed in `spark.local.dir` or the environment variable `SPARK_LOCAL_DIRS` . If no directories are explicitly specified then a default directory is created and configured appropriately. `emptyDir` volumes use the ephemeral storage feature of Kubernetes and do not persist beyond the life of the pod. From 7af4e358f3f4902cc9601e56c2662b8921a925d6 Mon Sep 17 00:00:00 2001 From: Martin Grund Date: Thu, 10 Aug 2023 08:41:13 +0900 Subject: [PATCH 09/12] [SPARK-44740][CONNECT] Support specifying `session_id` in SPARK_REMOTE connection string ### What changes were proposed in this pull request? To support cross-language session sharing in Spark connect, we need to be able to inject the session ID into the connection string because on the server side, the client-provided session ID is used already together with the user id. ``` SparkSession.builder.remote("sc://localhost/;session_id=abcdefg").getOrCreate() ``` ### Why are the changes needed? ease of use ### Does this PR introduce _any_ user-facing change? Adds a way to configure the Spark Connect connection string with `session_id` ### How was this patch tested? Added UT for the parameter. Closes #42415 from grundprinzip/SPARK-44740. Authored-by: Martin Grund Signed-off-by: Hyukjin Kwon --- .../connect/client/SparkConnectClient.scala | 22 ++++++++++++-- .../client/SparkConnectClientParser.scala | 3 ++ ...rkConnectClientBuilderParseTestSuite.scala | 4 +++ .../client/SparkConnectClientSuite.scala | 6 ++++ .../connect/docs/client-connection-string.md | 11 +++++++ python/pyspark/sql/connect/client/core.py | 30 ++++++++++++++++--- .../sql/tests/connect/client/test_client.py | 7 +++++ .../sql/tests/connect/test_connect_basic.py | 18 ++++++++++- 8 files changed, 94 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index a028df536cf88..637499f090ce4 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -56,7 +56,7 @@ private[sql] class SparkConnectClient( // Generate a unique session ID for this client. This UUID must be unique to allow // concurrent Spark sessions of the same user. If the channel is closed, creating // a new client will create a new session ID. - private[sql] val sessionId: String = UUID.randomUUID.toString + private[sql] val sessionId: String = configuration.sessionId.getOrElse(UUID.randomUUID.toString) private[client] val artifactManager: ArtifactManager = { new ArtifactManager(configuration, sessionId, bstub, stub) @@ -432,6 +432,7 @@ object SparkConnectClient { val PARAM_USE_SSL = "use_ssl" val PARAM_TOKEN = "token" val PARAM_USER_AGENT = "user_agent" + val PARAM_SESSION_ID = "session_id" } private def verifyURI(uri: URI): Unit = { @@ -463,6 +464,21 @@ object SparkConnectClient { this } + def sessionId(value: String): Builder = { + try { + UUID.fromString(value).toString + } catch { + case e: IllegalArgumentException => + throw new IllegalArgumentException( + "Parameter value 'session_id' must be a valid UUID format.", + e) + } + _configuration = _configuration.copy(sessionId = Some(value)) + this + } + + def sessionId: Option[String] = _configuration.sessionId + def userAgent: String = _configuration.userAgent def option(key: String, value: String): Builder = { @@ -490,6 +506,7 @@ object SparkConnectClient { case URIParams.PARAM_TOKEN => token(value) case URIParams.PARAM_USE_SSL => if (java.lang.Boolean.valueOf(value)) enableSsl() else disableSsl() + case URIParams.PARAM_SESSION_ID => sessionId(value) case _ => option(key, value) } } @@ -576,7 +593,8 @@ object SparkConnectClient { userAgent: String = DEFAULT_USER_AGENT, retryPolicy: GrpcRetryHandler.RetryPolicy = GrpcRetryHandler.RetryPolicy(), useReattachableExecute: Boolean = true, - interceptors: List[ClientInterceptor] = List.empty) { + interceptors: List[ClientInterceptor] = List.empty, + sessionId: Option[String] = None) { def userContext: proto.UserContext = { val builder = proto.UserContext.newBuilder() diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala index dda769dc2adb1..f873e1045bfcd 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala @@ -71,6 +71,9 @@ private[sql] object SparkConnectClientParser { case "--user_agent" :: tail => val (value, remainder) = extract("--user_agent", tail) parse(remainder, builder.userAgent(value)) + case "--session_id" :: tail => + val (value, remainder) = extract("--session_id", tail) + parse(remainder, builder.sessionId(value)) case "--option" :: tail => if (args.isEmpty) { throw new IllegalArgumentException("--option requires a key-value pair") diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBuilderParseTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBuilderParseTestSuite.scala index 2c6886d0386c5..1dc1fd567ec1a 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBuilderParseTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBuilderParseTestSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.connect.client +import java.util.UUID + import org.apache.spark.sql.connect.client.util.ConnectFunSuite /** @@ -46,6 +48,7 @@ class SparkConnectClientBuilderParseTestSuite extends ConnectFunSuite { argumentTest("user_id", "U1238", _.userId.get) argumentTest("user_name", "alice", _.userName.get) argumentTest("user_agent", "MY APP", _.userAgent) + argumentTest("session_id", UUID.randomUUID().toString, _.sessionId.get) test("Argument - remote") { val builder = @@ -55,6 +58,7 @@ class SparkConnectClientBuilderParseTestSuite extends ConnectFunSuite { assert(builder.token.contains("nahnah")) assert(builder.userId.contains("x127")) assert(builder.options === Map(("user_name", "Q"), ("param1", "x"))) + assert(builder.sessionId.isEmpty) } test("Argument - use_ssl") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 3436037809dbc..e483e0a7291bd 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -164,6 +164,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { client => { assert(client.configuration.host == "localhost") assert(client.configuration.port == 1234) + assert(client.sessionId != null) + // Must be able to parse the UUID + assert(UUID.fromString(client.sessionId) != null) }), TestPackURI( "sc://localhost/;", @@ -193,6 +196,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { TestPackURI("sc://host:123/;use_ssl=true", isCorrect = true), TestPackURI("sc://host:123/;token=mySecretToken", isCorrect = true), TestPackURI("sc://host:123/;token=", isCorrect = false), + TestPackURI("sc://host:123/;session_id=", isCorrect = false), + TestPackURI("sc://host:123/;session_id=abcdefgh", isCorrect = false), + TestPackURI(s"sc://host:123/;session_id=${UUID.randomUUID().toString}", isCorrect = true), TestPackURI("sc://host:123/;use_ssl=true;token=mySecretToken", isCorrect = true), TestPackURI("sc://host:123/;token=mySecretToken;use_ssl=true", isCorrect = true), TestPackURI("sc://host:123/;use_ssl=false;token=mySecretToken", isCorrect = false), diff --git a/connector/connect/docs/client-connection-string.md b/connector/connect/docs/client-connection-string.md index 6e5b0c80db7aa..ebab7cbff4fc1 100644 --- a/connector/connect/docs/client-connection-string.md +++ b/connector/connect/docs/client-connection-string.md @@ -91,6 +91,17 @@ sc://hostname:port/;param1=value;param2=value Default:
_SPARK_CONNECT_PYTHON
in the Python client
user_agent=my_data_query_app
+ + session_id + String + In addition to the user ID, the cache of Spark Sessions in the Spark Connect + server uses a session ID as the cache key. This option in the connection string + allows to provide this session ID to allow sharing Spark Sessions for the same users + for example across multiple languages. The value must be provided in a valid UUID + string format.
+ Default: A UUID generated randomly. +
session_id=550e8400-e29b-41d4-a716-446655440000
+ ## Examples diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index a7c3a92d3b1dc..5e6aacf5999a9 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -156,6 +156,7 @@ class ChannelBuilder: PARAM_TOKEN = "token" PARAM_USER_ID = "user_id" PARAM_USER_AGENT = "user_agent" + PARAM_SESSION_ID = "session_id" MAX_MESSAGE_LENGTH = 128 * 1024 * 1024 @staticmethod @@ -354,6 +355,22 @@ def get(self, key: str) -> Any: """ return self.params[key] + @property + def session_id(self) -> Optional[str]: + """ + Returns + ------- + The session_id extracted from the parameters of the connection string or `None` if not + specified. + """ + session_id = self.params.get(ChannelBuilder.PARAM_SESSION_ID, None) + if session_id is not None: + try: + uuid.UUID(session_id, version=4) + except ValueError as ve: + raise ValueError("Parameter value 'session_id' must be a valid UUID format.", ve) + return session_id + def toChannel(self) -> grpc.Channel: """ Applies the parameters of the connection string and creates a new @@ -628,10 +645,15 @@ def __init__( if retry_policy: self._retry_policy.update(retry_policy) - # Generate a unique session ID for this client. This UUID must be unique to allow - # concurrent Spark sessions of the same user. If the channel is closed, creating - # a new client will create a new session ID. - self._session_id = str(uuid.uuid4()) + if self._builder.session_id is None: + # Generate a unique session ID for this client. This UUID must be unique to allow + # concurrent Spark sessions of the same user. If the channel is closed, creating + # a new client will create a new session ID. + self._session_id = str(uuid.uuid4()) + else: + # Use the pre-defined session ID. + self._session_id = str(self._builder.session_id) + if self._builder.userId is not None: self._user_id = self._builder.userId elif user_id is not None: diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index 9276b88e153b8..9782add92f47d 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -16,6 +16,7 @@ # import unittest +import uuid from typing import Optional from pyspark.sql.connect.client import SparkConnectClient, ChannelBuilder @@ -88,6 +89,12 @@ def test_is_closed(self): client.close() self.assertTrue(client.is_closed) + def test_channel_builder_with_session(self): + dummy = str(uuid.uuid4()) + chan = ChannelBuilder(f"sc://foo/;session_id={dummy}") + client = SparkConnectClient(chan) + self.assertEqual(client._session_id, chan.session_id) + class MockService: # Simplest mock of the SparkConnectService. diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 0687fc9f31331..63b65ecce1a38 100644 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -23,6 +23,7 @@ import shutil import string import tempfile +import uuid from collections import defaultdict from pyspark.errors import ( @@ -76,7 +77,7 @@ from pyspark.sql.connect.dataframe import DataFrame as CDataFrame from pyspark.sql import functions as SF from pyspark.sql.connect import functions as CF - from pyspark.sql.connect.client.core import Retrying + from pyspark.sql.connect.client.core import Retrying, SparkConnectClient class SparkConnectSQLTestCase(ReusedConnectTestCase, SQLTestUtils, PandasOnSparkTestUtils): @@ -3522,6 +3523,21 @@ def test_metadata(self): md = chan.metadata() self.assertEqual([("param1", "120 21"), ("x-my-header", "abcd")], md) + def test_metadata(self): + id = str(uuid.uuid4()) + chan = ChannelBuilder(f"sc://host/;session_id={id}") + self.assertEqual(id, chan.session_id) + + with self.assertRaises(ValueError) as ve: + chan = ChannelBuilder("sc://host/;session_id=abcd") + SparkConnectClient(chan) + self.assertIn( + "Parameter value 'session_id' must be a valid UUID format.", str(ve.exception) + ) + + chan = ChannelBuilder("sc://host/") + self.assertIsNone(chan.session_id) + if __name__ == "__main__": from pyspark.sql.tests.connect.test_connect_basic import * # noqa: F401 From d27496eb3bf962981e37f989ba486d847745444f Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 10 Aug 2023 09:49:45 +0900 Subject: [PATCH 10/12] [SPARK-44747][CONNECT] Add missing SparkSession.Builder methods ### What changes were proposed in this pull request? This PR adds a couple methods to SparkSession.Builder: - `conf` - this group of methods allows you to set runtime configurations on the Spark Connect Session. - `master` - this is a no-op, it is only added for compatibility. - `appName` - this is a no-op, it is only added for compatibility. - `enableHiveSupport ` - this is a no-op, it is only added for compatibility. ### Why are the changes needed? We want to maximize compatiblity with the existing API in sql/core. ### Does this PR introduce _any_ user-facing change? Yes. It adds a couple of builder methods. ### How was this patch tested? Add tests to `SparkSessionSuite` and `SparkSessionE2ESuite`. Closes #42419 from hvanhovell/SPARK-44747. Authored-by: Herman van Hovell Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/SparkSession.scala | 91 ++++++++++++++++++- .../spark/sql/SparkSessionE2ESuite.scala | 46 ++++++++++ .../apache/spark/sql/SparkSessionSuite.scala | 10 ++ .../CheckConnectJvmClientCompatibility.scala | 6 -- 4 files changed, 146 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 7367ed153f7db..e902e04e24611 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -781,6 +781,7 @@ object SparkSession extends Logging { class Builder() extends Logging { private val builder = SparkConnectClient.builder() private var client: SparkConnectClient = _ + private[this] val options = new scala.collection.mutable.HashMap[String, String] def remote(connectionString: String): Builder = { builder.connectionString(connectionString) @@ -804,6 +805,84 @@ object SparkSession extends Logging { this } + /** + * Sets a config option. Options set using this method are automatically propagated to the + * Spark Connect session. Only runtime options are supported. + * + * @since 3.5.0 + */ + def config(key: String, value: String): Builder = synchronized { + options += key -> value + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to the + * Spark Connect session. Only runtime options are supported. + * + * @since 3.5.0 + */ + def config(key: String, value: Long): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to the + * Spark Connect session. Only runtime options are supported. + * + * @since 3.5.0 + */ + def config(key: String, value: Double): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to the + * Spark Connect session. Only runtime options are supported. + * + * @since 3.5.0 + */ + def config(key: String, value: Boolean): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config a map of options. Options set using this method are automatically propagated + * to the Spark Connect session. Only runtime options are supported. + * + * @since 3.5.0 + */ + def config(map: Map[String, Any]): Builder = synchronized { + map.foreach { kv: (String, Any) => + { + options += kv._1 -> kv._2.toString + } + } + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to both + * `SparkConf` and SparkSession's own configuration. + * + * @since 3.5.0 + */ + def config(map: java.util.Map[String, Any]): Builder = synchronized { + config(map.asScala.toMap) + } + + @deprecated("enableHiveSupport does not work in Spark Connect") + def enableHiveSupport(): Builder = this + + @deprecated("master does not work in Spark Connect, please use remote instead") + def master(master: String): Builder = this + + @deprecated("appName does not work in Spark Connect") + def appName(name: String): Builder = this + private def tryCreateSessionFromClient(): Option[SparkSession] = { if (client != null) { Option(new SparkSession(client, cleaner, planIdGenerator)) @@ -812,6 +891,12 @@ object SparkSession extends Logging { } } + private def applyOptions(session: SparkSession): Unit = { + options.foreach { case (key, value) => + session.conf.set(key, value) + } + } + /** * Build the [[SparkSession]]. * @@ -833,6 +918,7 @@ object SparkSession extends Logging { val session = tryCreateSessionFromClient() .getOrElse(SparkSession.this.create(builder.configuration)) setDefaultAndActiveSession(session) + applyOptions(session) session } @@ -842,7 +928,9 @@ object SparkSession extends Logging { * If a session exist with the same configuration that is returned instead of creating a new * session. * - * This method will update the default and/or active session if they are not set. + * This method will update the default and/or active session if they are not set. This method + * will always set the specified configuration options on the session, even when it is not + * newly created. * * @since 3.5.0 */ @@ -850,6 +938,7 @@ object SparkSession extends Logging { val session = tryCreateSessionFromClient() .getOrElse(sessions.get(builder.configuration)) setDefaultAndActiveSession(session) + applyOptions(session) session } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala index 86deae982a5d0..490bdf9cd86ec 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala @@ -249,4 +249,50 @@ class SparkSessionE2ESuite extends RemoteSparkSession { } assert(e.getMessage contains "OPERATION_CANCELED") } + + test("option propagation") { + val remote = s"sc://localhost:$serverPort" + val session1 = SparkSession + .builder() + .remote(remote) + .config("foo", 12L) + .config("bar", value = true) + .config("bob", 12.0) + .config("heading", "north") + .getOrCreate() + assert(session1.conf.get("foo") == "12") + assert(session1.conf.get("bar") == "true") + assert(session1.conf.get("bob") == String.valueOf(12.0)) + assert(session1.conf.get("heading") == "north") + + // Check if new options are applied to an existing session. + val session2 = SparkSession + .builder() + .remote(remote) + .config("heading", "south") + .getOrCreate() + assert(session2 == session1) + assert(session2.conf.get("heading") == "south") + + // Create a completely different session, confs are not support to leak. + val session3 = SparkSession + .builder() + .remote(remote) + .config(Map("foo" -> "13", "baar" -> "false", "heading" -> "east")) + .create() + assert(session3 != session1) + assert(session3.conf.get("foo") == "13") + assert(session3.conf.get("baar") == "false") + assert(session3.conf.getOption("bob").isEmpty) + assert(session3.conf.get("heading") == "east") + + // Try to set a static conf. + intercept[Exception] { + SparkSession + .builder() + .remote(remote) + .config("spark.sql.globalTempDatabase", "not_gonna_happen") + .create() + } + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala index 2d7ded2d68858..4aa8b4360eebd 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala @@ -251,4 +251,14 @@ class SparkSessionSuite extends ConnectFunSuite { executor.shutdown() } } + + test("deprecated methods") { + SparkSession + .builder() + .master("yayay") + .appName("bob") + .enableHiveSupport() + .create() + .close() + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 4439a5f3e2adb..3fc02d7c397f0 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -223,14 +223,8 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.SparkSession#implicits._sqlContext"), // SparkSession#Builder - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.SparkSession#Builder.appName"), ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.SparkSession#Builder.config"), - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.SparkSession#Builder.master"), - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.SparkSession#Builder.enableHiveSupport"), ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.SparkSession#Builder.withExtensions"), From 3b3e0fdc61cf659bb97f94d9b12b8dcdad999e62 Mon Sep 17 00:00:00 2001 From: Zhyhimont Dmitry Date: Thu, 10 Aug 2023 11:05:11 +0900 Subject: [PATCH 11/12] [SPARK-42621][PS] Add inclusive parameter for pd.date_range ### What changes were proposed in this pull request? Add inclusive parameter for pd.date_range to support the pandas 2.0.0 ### Why are the changes needed? When pandas 2.0.0 is released, we should match the behavior in pandas API on Spark. ### Does this PR introduce any user-facing change? yes, the API changes Before: ps.date_range(start='2017-01-01', end='2017-01-04', closed=None) After: ps.date_range(start='2017-01-01', end='2017-01-04', inclusive="both") ### How was this patch tested? Unit tests were updated Closes #40665 from dzhigimont/SPARK-42621_ZH. Lead-authored-by: Zhyhimont Dmitry Co-authored-by: Zhyhimont Dmitry Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/namespace.py | 32 +++++++++++++++++-- python/pyspark/pandas/tests/test_namespace.py | 25 +++++++++++++++ 2 files changed, 55 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/namespace.py b/python/pyspark/pandas/namespace.py index ba93e5a3ee506..fddf1bec63fcc 100644 --- a/python/pyspark/pandas/namespace.py +++ b/python/pyspark/pandas/namespace.py @@ -1751,8 +1751,6 @@ def pandas_to_datetime( ) -# TODO(SPARK-42621): Add `inclusive` parameter. -# See https://github.com/pandas-dev/pandas/issues/40245 def date_range( start: Union[str, Any] = None, end: Union[str, Any] = None, @@ -1761,6 +1759,7 @@ def date_range( tz: Optional[Union[str, tzinfo]] = None, normalize: bool = False, name: Optional[str] = None, + inclusive: str = "both", **kwargs: Any, ) -> DatetimeIndex: """ @@ -1784,6 +1783,11 @@ def date_range( Normalize start/end dates to midnight before generating date range. name : str, default None Name of the resulting DatetimeIndex. + inclusive : {"both", "neither", "left", "right"}, default "both" + Include boundaries; Whether to set each bound as closed or open. + + .. versionadded:: 4.0.0 + **kwargs For compatibility. Has no effect on the result. @@ -1867,6 +1871,29 @@ def date_range( DatetimeIndex(['2018-01-31', '2018-04-30', '2018-07-31', '2018-10-31', '2019-01-31'], dtype='datetime64[ns]', freq=None) + + `inclusive` controls whether to include `start` and `end` that are on the + boundary. The default includes boundary points on either end. + + >>> ps.date_range( + ... start='2017-01-01', end='2017-01-04', inclusive="both" + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-01', '2017-01-02', '2017-01-03', '2017-01-04'], + dtype='datetime64[ns]', freq=None) + + Use ``inclusive='left'`` to exclude `end` if it falls on the boundary. + + >>> ps.date_range( + ... start='2017-01-01', end='2017-01-04', inclusive='left' + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-01', '2017-01-02', '2017-01-03'], dtype='datetime64[ns]', freq=None) + + Use ``inclusive='right'`` to exclude `start` if it falls on the boundary. + + >>> ps.date_range( + ... start='2017-01-01', end='2017-01-04', inclusive='right' + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-02', '2017-01-03', '2017-01-04'], dtype='datetime64[ns]', freq=None) """ assert freq not in ["N", "ns"], "nanoseconds is not supported" assert tz is None, "Localized DatetimeIndex is not supported" @@ -1882,6 +1909,7 @@ def date_range( tz=tz, normalize=normalize, name=name, + inclusive=inclusive, **kwargs, ) ), diff --git a/python/pyspark/pandas/tests/test_namespace.py b/python/pyspark/pandas/tests/test_namespace.py index d1d1e1af9354d..5c20204671740 100644 --- a/python/pyspark/pandas/tests/test_namespace.py +++ b/python/pyspark/pandas/tests/test_namespace.py @@ -221,6 +221,31 @@ def test_date_range(self): pd.date_range(start="1/1/2018", periods=5, freq=pd.offsets.MonthEnd(3)), ) + self.assert_eq( + ps.date_range(start="2017-01-01", end="2017-01-04", inclusive="left"), + pd.date_range(start="2017-01-01", end="2017-01-04", inclusive="left"), + ) + + self.assert_eq( + ps.date_range(start="2017-01-01", end="2017-01-04", inclusive="right"), + pd.date_range(start="2017-01-01", end="2017-01-04", inclusive="right"), + ) + + self.assert_eq( + ps.date_range(start="2017-01-01", end="2017-01-04", inclusive="both"), + pd.date_range(start="2017-01-01", end="2017-01-04", inclusive="both"), + ) + + self.assert_eq( + ps.date_range(start="2017-01-01", end="2017-01-04", inclusive="neither"), + pd.date_range(start="2017-01-01", end="2017-01-04", inclusive="neither"), + ) + + with self.assertRaisesRegex( + ValueError, "Inclusive has to be either 'both', 'neither', 'left' or 'right'" + ): + ps.date_range(start="2017-01-01", end="2017-01-04", inclusive="test") + self.assertRaises( AssertionError, lambda: ps.date_range(start="1/1/2018", periods=5, tz="Asia/Tokyo") ) From 1846991b20a07e95b5e0296108565216841dfe52 Mon Sep 17 00:00:00 2001 From: Sandip Agarwala <131817656+sandip-db@users.noreply.github.com> Date: Thu, 10 Aug 2023 11:23:40 +0900 Subject: [PATCH 12/12] [SPARK-44732][SQL] Built-in XML data source support ### What changes were proposed in this pull request? XML is a widely used data format. An external spark-xml package (https://github.com/databricks/spark-xml) is available to read and write XML data in spark. Making spark-xml built-in will provide a better user experience for Spark SQL and structured streaming. The proposal is to inline code from spark-xml package. The PR has the following commits: i) The first commit has the following: - Copy of spark-xml src files. - Update license - Scala style and format fixes - Change AnyFunSuite to SparkFunSuite ii) Miscellaneous import and scala style fixes. iii) Add library dependencies iv) Resource file path fixes and change AnyFunSuite to SharedSparkSession or SQLTestUtils v) Exclude XML test resource files from license check vi) Change import from scala.jdk.Collections to scala.collection.JavaConverters ### Why are the changes needed? Built-in support for XML data source would provide better user experience than having to import an external package. ### Does this PR introduce _any_ user-facing change? Yes, Add built-in support for XML data source. ### How was this patch tested? Tested the new unit-tests that came with the imported spark-xml package. Also ran ./dev/run-test Closes #41832 from sandip-db/spark-xml-master. Authored-by: Sandip Agarwala <131817656+sandip-db@users.noreply.github.com> Signed-off-by: Hyukjin Kwon --- dev/.rat-excludes | 1 + dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 + pom.xml | 12 + sql/core/pom.xml | 8 + ...pache.spark.sql.sources.DataSourceRegister | 1 + .../datasources/xml/DefaultSource.scala | 113 + .../datasources/xml/XmlDataToCatalyst.scala | 62 + .../datasources/xml/XmlInputFormat.scala | 341 ++ .../datasources/xml/XmlOptions.scala | 85 + .../execution/datasources/xml/XmlReader.scala | 205 + .../datasources/xml/XmlRelation.scala | 86 + .../execution/datasources/xml/functions.scala | 42 + .../execution/datasources/xml/package.scala | 162 + .../xml/parsers/StaxXmlGenerator.scala | 159 + .../xml/parsers/StaxXmlParser.scala | 375 ++ .../xml/parsers/StaxXmlParserUtils.scala | 179 + .../datasources/xml/util/InferSchema.scala | 336 ++ .../xml/util/PartialResultException.scala | 29 + .../datasources/xml/util/TypeCast.scala | 297 ++ .../datasources/xml/util/ValidatorUtil.scala | 55 + .../datasources/xml/util/XSDToSchema.scala | 280 ++ .../datasources/xml/util/XmlFile.scala | 163 + .../datasources/xml/JavaXmlSuite.java | 111 + .../xml-resources/ages-mixed-types.xml | 15 + .../xml-resources/ages-with-spaces.xml | 20 + .../test-data/xml-resources/ages.xml | 14 + .../attributesStartWithNewLine.xml | 11 + .../attributesStartWithNewLineCR.xml | 1 + .../attributesStartWithNewLineLF.xml | 11 + .../test-data/xml-resources/basket.xml | 12 + .../test-data/xml-resources/basket.xsd | 17 + .../xml-resources/basket_invalid.xml | 14 + .../books-attributes-in-no-child.xml | 75 + .../books-complicated-null-attribute.xml | 60 + .../xml-resources/books-complicated.xml | 60 + .../books-malformed-attributes.xml | 43 + .../xml-resources/books-namespaces.xml | 12 + .../xml-resources/books-nested-array.xml | 130 + .../xml-resources/books-nested-object.xml | 144 + .../books-unicode-in-tag-name.xml | 24 + .../test-data/xml-resources/books.xml | 136 + .../xml-resources/cars-attribute.xml | 9 + .../xml-resources/cars-iso-8859-1.xml | 21 + .../xml-resources/cars-malformed.xml | 20 + .../cars-mixed-attr-no-child.xml | 25 + .../xml-resources/cars-no-indentation.xml | 2 + .../cars-unbalanced-elements.xml | 19 + .../test-data/xml-resources/cars.xml | 21 + .../test-data/xml-resources/cars.xml.bz2 | Bin 0 -> 229 bytes .../test-data/xml-resources/cars.xml.gz | Bin 0 -> 210 bytes .../test-data/xml-resources/catalog.xsd | 41 + .../test-data/xml-resources/choice.xsd | 12 + .../complex-content-extension.xsd | 25 + .../datatypes-valid-and-invalid.xml | 31 + .../test-data/xml-resources/date.xml | 5 + .../decimal-with-restriction.xsd | 18 + .../test-data/xml-resources/empty.xml | 0 .../xml-resources/feed-with-spaces.xml | 15 + .../xml-resources/fias_house.large.xml | 3621 +++++++++++++++++ .../xml-resources/fias_house.large.xml.bz2 | Bin 0 -> 30761 bytes .../xml-resources/fias_house.large.xml.gz | Bin 0 -> 8568 bytes .../test-data/xml-resources/fias_house.xml | 182 + .../xml-resources/fias_house.xml.bz2 | Bin 0 -> 4571 bytes .../test-data/xml-resources/fias_house.xml.gz | Bin 0 -> 5069 bytes .../xml-resources/gps-empty-field.xml | 20 + .../xml-resources/include-example/first.xsd | 5 + .../xml-resources/include-example/second.xsd | 15 + .../test-data/xml-resources/long.xsd | 10 + .../manual_schema_corrupt_record.xml | 30 + .../test-data/xml-resources/map-attribute.xml | 7 + .../xml-resources/mixed_children.xml | 5 + .../xml-resources/mixed_children_2.xml | 5 + .../mixed_children_as_string.xml | 9 + ...ent-with-attributes-and-name-of-parent.xml | 5 + .../nested-element-with-name-of-parent.xml | 5 + .../xml-resources/null-empty-string.xml | 7 + .../xml-resources/null-nested-struct-2.xml | 49 + .../xml-resources/null-nested-struct.xml | 20 + .../xml-resources/null-numbers-2.xml | 6 + .../test-data/xml-resources/null-numbers.xml | 15 + .../test-data/xml-resources/processing.xml | 6 + .../test-data/xml-resources/ref-attribute.xsd | 19 + .../xml-resources/self-closing-tag.xml | 6 + .../xml-resources/simple-nested-objects.xml | 14 + .../struct_with_optional_child.xml | 8 + .../test-data/xml-resources/textColumn.xml | 18 + .../test-data/xml-resources/time.xml | 7 + .../xml-resources/topics-namespaces.xml | 7 + .../test-data/xml-resources/twoelements.xsd | 5 + .../test-data/xml-resources/unclosed_tag.xml | 4 + .../xml-resources/whitespace_error.xml | 1 + .../test-data/xml-resources/xsany.xsd | 37 + .../execution/datasources/xml/TestUtils.scala | 37 + .../xml/XmlPartitioningSuite.scala | 75 + .../execution/datasources/xml/XmlSuite.scala | 1549 +++++++ .../xml/parsers/StaxXmlGeneratorSuite.scala | 78 + .../xml/parsers/StaxXmlParserUtilsSuite.scala | 94 + .../datasources/xml/util/TypeCastSuite.scala | 236 ++ .../xml/util/XSDToSchemaSuite.scala | 186 + .../datasources/xml/util/XmlFileSuite.scala | 69 + 100 files changed, 10639 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/DefaultSource.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataToCatalyst.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlOptions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/functions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/package.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParser.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtils.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/InferSchema.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/PartialResultException.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCast.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/ValidatorUtil.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchema.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFile.scala create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/xml/JavaXmlSuite.java create mode 100644 sql/core/src/test/resources/test-data/xml-resources/ages-mixed-types.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/ages-with-spaces.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/ages.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLine.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineCR.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineLF.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/basket.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/basket.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/basket_invalid.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-attributes-in-no-child.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-complicated-null-attribute.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-complicated.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-malformed-attributes.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-namespaces.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-nested-array.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-nested-object.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books-unicode-in-tag-name.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/books.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-attribute.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-iso-8859-1.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-malformed.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-mixed-attr-no-child.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-no-indentation.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars-unbalanced-elements.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars.xml.bz2 create mode 100644 sql/core/src/test/resources/test-data/xml-resources/cars.xml.gz create mode 100644 sql/core/src/test/resources/test-data/xml-resources/catalog.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/choice.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/complex-content-extension.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/datatypes-valid-and-invalid.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/date.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/decimal-with-restriction.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/empty.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/feed-with-spaces.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.bz2 create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.gz create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.bz2 create mode 100644 sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.gz create mode 100644 sql/core/src/test/resources/test-data/xml-resources/gps-empty-field.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/include-example/first.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/include-example/second.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/long.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/manual_schema_corrupt_record.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/map-attribute.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/mixed_children.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/mixed_children_2.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/mixed_children_as_string.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/nested-element-with-attributes-and-name-of-parent.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/nested-element-with-name-of-parent.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/null-empty-string.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/null-nested-struct-2.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/null-nested-struct.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/null-numbers-2.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/null-numbers.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/processing.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/ref-attribute.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/self-closing-tag.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/simple-nested-objects.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/struct_with_optional_child.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/textColumn.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/time.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/topics-namespaces.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/twoelements.xsd create mode 100644 sql/core/src/test/resources/test-data/xml-resources/unclosed_tag.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/whitespace_error.xml create mode 100644 sql/core/src/test/resources/test-data/xml-resources/xsany.xsd create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestUtils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGeneratorSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtilsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCastSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchemaSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFileSuite.scala diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 16e0e3e30c9e5..18d4cbcd53cd6 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -145,3 +145,4 @@ empty.proto .*\.proto.bin LimitedInputStream.java TimSort.java +xml-resources/* diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 416753ab2010c..2a5a6741bdd44 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -245,9 +245,11 @@ super-csv/2.2.0//super-csv-2.2.0.jar threeten-extra/1.7.1//threeten-extra-1.7.1.jar tink/1.9.0//tink-1.9.0.jar transaction-api/1.1//transaction-api-1.1.jar +txw2/3.0.2//txw2-3.0.2.jar univocity-parsers/2.9.1//univocity-parsers-2.9.1.jar wildfly-openssl/1.1.3.Final//wildfly-openssl-1.1.3.Final.jar xbean-asm9-shaded/4.23//xbean-asm9-shaded-4.23.jar +xmlschema-core/2.3.0//xmlschema-core-2.3.0.jar xz/1.9//xz-1.9.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper-jute/3.6.3//zookeeper-jute-3.6.3.jar diff --git a/pom.xml b/pom.xml index 624df0c314a0e..30357f1b27fad 100644 --- a/pom.xml +++ b/pom.xml @@ -182,6 +182,8 @@ 1.9.13 2.15.2 2.15.2 + 2.3.0 + 3.0.2 1.1.10.3 3.0.3 1.16.0 @@ -986,6 +988,16 @@ jackson-module-jaxb-annotations ${fasterxml.jackson.version} + + org.apache.ws.xmlschema + xmlschema-core + ${ws.xmlschema.version} + + + org.glassfish.jaxb + txw2 + ${org.glassfish.jaxb.txw2.version} + org.glassfish.jersey.core jersey-server diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 709734b541f01..bf3caf58fe276 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -143,6 +143,14 @@ com.fasterxml.jackson.core jackson-databind + + org.apache.ws.xmlschema + xmlschema-core + + + org.glassfish.jaxb + txw2 + org.apache.xbean xbean-asm9-shaded diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 1365134641758..db418b7363692 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -22,6 +22,7 @@ org.apache.spark.sql.execution.datasources.noop.NoopDataSource org.apache.spark.sql.execution.datasources.orc.OrcFileFormat org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 org.apache.spark.sql.execution.datasources.v2.text.TextDataSourceV2 +org.apache.spark.sql.execution.datasources.xml.DefaultSource org.apache.spark.sql.execution.streaming.ConsoleSinkProvider org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/DefaultSource.scala new file mode 100644 index 0000000000000..a1da349321b42 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/DefaultSource.scala @@ -0,0 +1,113 @@ +/* + * 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.execution.datasources.xml + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.execution.datasources.xml.util.XmlFile +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType + +/** + * Provides access to XML data from pure SQL statements (i.e. for users of the + * JDBC server). + */ +class DefaultSource + extends RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider + with DataSourceRegister { + + /** + * Short alias for spark-xml data source. + */ + override def shortName(): String = "xml" + + private def checkPath(parameters: Map[String, String]): String = { + parameters.getOrElse("path", + throw new IllegalArgumentException("'path' must be specified for XML data.")) + } + + /** + * Creates a new relation for data store in XML given parameters. + * Parameters have to include 'path'. + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + createRelation(sqlContext, parameters, null) + } + + /** + * Creates a new relation for data store in XML given parameters and user supported schema. + * Parameters have to include 'path'. + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): XmlRelation = { + val path = checkPath(parameters) + // We need the `charset` and `rowTag` before creating the relation. + val (charset, rowTag) = { + val options = XmlOptions(parameters) + (options.charset, options.rowTag) + } + + val paramsWithTZ = + sqlContext.sparkContext.getConf.getOption("spark.sql.session.timeZone") match { + case Some(tz) => parameters.updated("timezone", tz) + case None => parameters + } + + XmlRelation( + () => XmlFile.withCharset(sqlContext.sparkContext, path, charset, rowTag), + Some(path), + paramsWithTZ, + schema)(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + val path = checkPath(parameters) + val filesystemPath = new Path(path) + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + val doSave = if (fs.exists(filesystemPath)) { + mode match { + case SaveMode.Append => + throw new IllegalArgumentException( + s"Append mode is not supported by ${this.getClass.getCanonicalName}") + case SaveMode.Overwrite => + fs.delete(filesystemPath, true) + true + case SaveMode.ErrorIfExists => + throw new IllegalArgumentException(s"path $path already exists.") + case SaveMode.Ignore => false + } + } else { + true + } + if (doSave) { + // Only save data when the save mode is not ignore. + XmlFile.saveAsXmlFile(data, filesystemPath.toString, parameters) + } + createRelation(sqlContext, parameters, data.schema) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataToCatalyst.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataToCatalyst.scala new file mode 100644 index 0000000000000..95e9743d302df --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataToCatalyst.scala @@ -0,0 +1,62 @@ +/* + * 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.execution.datasources.xml + +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlParser +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +case class XmlDataToCatalyst( + child: Expression, + schema: DataType, + options: XmlOptions) + extends UnaryExpression with CodegenFallback with ExpectsInputTypes { + + override lazy val dataType: DataType = schema + + @transient + lazy val rowSchema: StructType = schema match { + case st: StructType => st + case ArrayType(st: StructType, _) => st + } + + override def nullSafeEval(xml: Any): Any = xml match { + case string: UTF8String => + CatalystTypeConverters.convertToCatalyst( + StaxXmlParser.parseColumn(string.toString, rowSchema, options)) + case string: String => + StaxXmlParser.parseColumn(string, rowSchema, options) + case arr: GenericArrayData => + CatalystTypeConverters.convertToCatalyst( + arr.array.map(s => StaxXmlParser.parseColumn(s.toString, rowSchema, options))) + case arr: Array[_] => + arr.map(s => StaxXmlParser.parseColumn(s.toString, rowSchema, options)) + case _ => null + } + + override def inputTypes: Seq[DataType] = schema match { + case _: StructType => Seq(StringType) + case ArrayType(_: StructType, _) => Seq(ArrayType(StringType)) + } + + // Overrides, in Spark 3.2.0+ + protected def withNewChildInternal(newChild: Expression): XmlDataToCatalyst = copy(newChild) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala new file mode 100644 index 0000000000000..40c32d3aa7628 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala @@ -0,0 +1,341 @@ +/* + * 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.execution.datasources.xml + +import java.io.{InputStream, InputStreamReader, IOException, Reader} +import java.nio.ByteBuffer +import java.nio.charset.Charset + +import org.apache.commons.io.input.CountingInputStream +import org.apache.hadoop.fs.Seekable +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.io.compress._ +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.{FileSplit, TextInputFormat} + +/** + * Reads records that are delimited by a specific start/end tag. + */ +class XmlInputFormat extends TextInputFormat { + + override def createRecordReader( + split: InputSplit, + context: TaskAttemptContext): RecordReader[LongWritable, Text] = { + new XmlRecordReader + } +} + +object XmlInputFormat { + /** configuration key for start tag */ + val START_TAG_KEY: String = "xmlinput.start" + /** configuration key for end tag */ + val END_TAG_KEY: String = "xmlinput.end" + /** configuration key for encoding type */ + val ENCODING_KEY: String = "xmlinput.encoding" +} + +/** + * XMLRecordReader class to read through a given xml document to output xml blocks as records + * as specified by the start tag and end tag. + * + * This implementation is ultimately loosely based on LineRecordReader in Hadoop. + */ +private[xml] class XmlRecordReader extends RecordReader[LongWritable, Text] { + + private var startTag: String = _ + private var currentStartTag: String = _ + private var endTag: String = _ + private var currentKey: LongWritable = _ + private var currentValue: Text = _ + private var start: Long = _ + private var end: Long = _ + private var reader: Reader = _ + private var filePosition: Seekable = _ + private var countingIn: CountingInputStream = _ + private var readerLeftoverCharFn: () => Boolean = _ + private var readerByteBuffer: ByteBuffer = _ + private var decompressor: Decompressor = _ + private var buffer = new StringBuilder() + + override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = { + val fileSplit = split.asInstanceOf[FileSplit] + val conf = context.getConfiguration + val charset = + Charset.forName(conf.get(XmlInputFormat.ENCODING_KEY, XmlOptions.DEFAULT_CHARSET)) + startTag = conf.get(XmlInputFormat.START_TAG_KEY) + endTag = conf.get(XmlInputFormat.END_TAG_KEY) + start = fileSplit.getStart + end = start + fileSplit.getLength + + // open the file and seek to the start of the split + val path = fileSplit.getPath + val fs = path.getFileSystem(conf) + val fsin = fs.open(fileSplit.getPath) + + var in: InputStream = null + val codec = new CompressionCodecFactory(conf).getCodec(path) + if (null != codec) { + decompressor = CodecPool.getDecompressor(codec) + codec match { + case sc: SplittableCompressionCodec => + val cIn = sc.createInputStream( + fsin, + decompressor, + start, + end, + SplittableCompressionCodec.READ_MODE.BYBLOCK) + start = cIn.getAdjustedStart + end = cIn.getAdjustedEnd + in = cIn + filePosition = cIn + case c: CompressionCodec => + if (start != 0) { + // So we have a split that is only part of a file stored using + // a Compression codec that cannot be split. + throw new IOException("Cannot seek in " + + codec.getClass.getSimpleName + " compressed stream") + } + val cIn = c.createInputStream(fsin, decompressor) + in = cIn + filePosition = fsin + } + } else { + fsin.seek(start) + countingIn = new CountingInputStream(fsin) + in = countingIn + // don't use filePosition in this case. We have to count bytes read manually + } + + reader = new InputStreamReader(in, charset) + + if (codec == null) { + // Hack: in the uncompressed case (see more below), we must know how much the + // InputStreamReader has buffered but not processed + // to accurately assess how many bytes have been processed + val sdField = reader.getClass.getDeclaredField("sd") + sdField.setAccessible(true) + val sd = sdField.get(reader) + val readerLeftoverCharField = sd.getClass.getDeclaredField("haveLeftoverChar") + readerLeftoverCharField.setAccessible(true) + readerLeftoverCharFn = () => { readerLeftoverCharField.get(sd).asInstanceOf[Boolean] } + val bbField = sd.getClass.getDeclaredField("bb") + bbField.setAccessible(true) + readerByteBuffer = bbField.get(sd).asInstanceOf[ByteBuffer] + } + } + + /** + * Tries to determine how many bytes of the underlying split have been read. There are two + * distinct cases. + * + * For compressed input, it attempts to read the current position read in the compressed input + * stream. This logic is copied from LineRecordReader, essentially. + * + * For uncompressed input, it counts the number of bytes read directly from the split. It + * further compensates for the fact that the intervening InputStreamReader buffers input and + * accounts for data it has read but not yet returned. + */ + private def getFilePosition(): Long = { + // filePosition != null when input is compressed + if (filePosition != null) { + filePosition.getPos + } else { + start + countingIn.getByteCount - + readerByteBuffer.remaining() - + (if (readerLeftoverCharFn()) 1 else 0) + } + } + + override def nextKeyValue: Boolean = { + currentKey = new LongWritable + currentValue = new Text + next(currentKey, currentValue) + } + + /** + * Finds the start of the next record. + * It treats data from `startTag` and `endTag` as a record. + * + * @param key the current key that will be written + * @param value the object that will be written + * @return whether it reads successfully + */ + private def next(key: LongWritable, value: Text): Boolean = { + if (readUntilStartElement()) { + try { + buffer.append(currentStartTag) + // Don't check whether the end element was found. Even if not, return everything + // that was read, which will invariably cause a parse error later + readUntilEndElement(currentStartTag.endsWith(">")) + key.set(getFilePosition()) + value.set(buffer.toString()) + return true + } finally { + buffer = new StringBuilder() + } + } + false + } + + private def readUntilStartElement(): Boolean = { + currentStartTag = startTag + var i = 0 + while (true) { + val cOrEOF = reader.read() + if (cOrEOF == -1 || (i == 0 && getFilePosition() > end)) { + // End of file or end of split. + return false + } + val c = cOrEOF.toChar + if (c == startTag(i)) { + if (i >= startTag.length - 1) { + // Found start tag. + return true + } + // else in start tag + i += 1 + } else { + // if doesn't match the closing angle bracket, check if followed by attributes + if (i == (startTag.length - 1) && Character.isWhitespace(c)) { + // Found start tag with attributes. Remember to write with following whitespace + // char, not angle bracket + currentStartTag = startTag.dropRight(1) + c + return true + } + // else not in start tag + i = 0 + } + } + // Unreachable. + false + } + + private def readUntilEndElement(startTagClosed: Boolean): Boolean = { + // Index into the start or end tag that has matched so far + var si = 0 + var ei = 0 + // How many other start tags enclose the one that's started already? + var depth = 0 + // Previously read character + var prevC = '\u0000' + + // The current start tag already found may or may not have terminated with + // a '>' as it may have attributes we read here. If not, we search for + // a self-close tag, but only until a non-self-closing end to the start + // tag is found + var canSelfClose = !startTagClosed + + while (true) { + + val cOrEOF = reader.read() + if (cOrEOF == -1) { + // End of file (ignore end of split). + return false + } + + val c = cOrEOF.toChar + buffer.append(c) + + if (c == '>' && prevC != '/') { + canSelfClose = false + } + + // Still matching a start tag? + if (c == startTag(si)) { + // Still also matching an end tag? + if (c == endTag(ei)) { + // In start tag or end tag. + si += 1 + ei += 1 + } else { + if (si >= startTag.length - 1) { + // Found start tag. + si = 0 + ei = 0 + depth += 1 + } else { + // In start tag. + si += 1 + ei = 0 + } + } + } else if (c == endTag(ei)) { + if (ei >= endTag.length - 1) { + if (depth == 0) { + // Found closing end tag. + return true + } + // else found nested end tag. + si = 0 + ei = 0 + depth -= 1 + } else { + // In end tag. + si = 0 + ei += 1 + } + } else if (c == '>' && prevC == '/' && canSelfClose) { + if (depth == 0) { + // found a self-closing tag (end tag) + return true + } + // else found self-closing nested tag (end tag) + si = 0 + ei = 0 + depth -= 1 + } else if (si == (startTag.length - 1) && Character.isWhitespace(c)) { + // found a start tag with attributes + si = 0 + ei = 0 + depth += 1 + } else { + // Not in start tag or end tag. + si = 0 + ei = 0 + } + prevC = c + } + // Unreachable. + false + } + + override def getProgress: Float = { + if (start == end) { + 0.0f + } else { + math.min(1.0f, (getFilePosition() - start) / (end - start).toFloat) + } + } + + override def getCurrentKey: LongWritable = currentKey + + override def getCurrentValue: Text = currentValue + + def close(): Unit = { + try { + if (reader != null) { + reader.close() + reader = null + } + } finally { + if (decompressor != null) { + CodecPool.returnDecompressor(decompressor) + decompressor = null + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlOptions.scala new file mode 100644 index 0000000000000..ad930d74ffb3d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlOptions.scala @@ -0,0 +1,85 @@ +/* + * 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.execution.datasources.xml + +import java.nio.charset.StandardCharsets + +import org.apache.spark.sql.catalyst.util.{ParseMode, PermissiveMode} + +/** + * Options for the XML data source. + */ +private[xml] class XmlOptions( + @transient private val parameters: Map[String, String]) + extends Serializable { + + def this() = this(Map.empty) + + val charset = parameters.getOrElse("charset", XmlOptions.DEFAULT_CHARSET) + val codec = parameters.get("compression").orElse(parameters.get("codec")).orNull + val rowTag = parameters.getOrElse("rowTag", XmlOptions.DEFAULT_ROW_TAG) + require(rowTag.nonEmpty, "'rowTag' option should not be empty string.") + require(!rowTag.startsWith("<") && !rowTag.endsWith(">"), + "'rowTag' should not include angle brackets") + val rootTag = parameters.getOrElse("rootTag", XmlOptions.DEFAULT_ROOT_TAG) + require(!rootTag.startsWith("<") && !rootTag.endsWith(">"), + "'rootTag' should not include angle brackets") + val declaration = parameters.getOrElse("declaration", XmlOptions.DEFAULT_DECLARATION) + require(!declaration.startsWith("<") && !declaration.endsWith(">"), + "'declaration' should not include angle brackets") + val arrayElementName = parameters.getOrElse("arrayElementName", + XmlOptions.DEFAULT_ARRAY_ELEMENT_NAME) + val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) + require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") + val excludeAttributeFlag = parameters.get("excludeAttribute").map(_.toBoolean).getOrElse(false) + val treatEmptyValuesAsNulls = + parameters.get("treatEmptyValuesAsNulls").map(_.toBoolean).getOrElse(false) + val attributePrefix = + parameters.getOrElse("attributePrefix", XmlOptions.DEFAULT_ATTRIBUTE_PREFIX) + val valueTag = parameters.getOrElse("valueTag", XmlOptions.DEFAULT_VALUE_TAG) + require(valueTag.nonEmpty, "'valueTag' option should not be empty string.") + require(valueTag != attributePrefix, + "'valueTag' and 'attributePrefix' options should not be the same.") + val nullValue = parameters.getOrElse("nullValue", XmlOptions.DEFAULT_NULL_VALUE) + val columnNameOfCorruptRecord = + parameters.getOrElse("columnNameOfCorruptRecord", "_corrupt_record") + val ignoreSurroundingSpaces = + parameters.get("ignoreSurroundingSpaces").map(_.toBoolean).getOrElse(false) + val parseMode = ParseMode.fromString(parameters.getOrElse("mode", PermissiveMode.name)) + val inferSchema = parameters.get("inferSchema").map(_.toBoolean).getOrElse(true) + val rowValidationXSDPath = parameters.get("rowValidationXSDPath").orNull + val wildcardColName = + parameters.getOrElse("wildcardColName", XmlOptions.DEFAULT_WILDCARD_COL_NAME) + val ignoreNamespace = parameters.get("ignoreNamespace").map(_.toBoolean).getOrElse(false) + val timestampFormat = parameters.get("timestampFormat") + val timezone = parameters.get("timezone") + val dateFormat = parameters.get("dateFormat") +} + +private[xml] object XmlOptions { + val DEFAULT_ATTRIBUTE_PREFIX = "_" + val DEFAULT_VALUE_TAG = "_VALUE" + val DEFAULT_ROW_TAG = "ROW" + val DEFAULT_ROOT_TAG = "ROWS" + val DEFAULT_DECLARATION = "version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"" + val DEFAULT_ARRAY_ELEMENT_NAME = "item" + val DEFAULT_CHARSET: String = StandardCharsets.UTF_8.name + val DEFAULT_NULL_VALUE: String = null + val DEFAULT_WILDCARD_COL_NAME = "xs_any" + + def apply(parameters: Map[String, String]): XmlOptions = new XmlOptions(parameters) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlReader.scala new file mode 100644 index 0000000000000..80be4fb0072f6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlReader.scala @@ -0,0 +1,205 @@ +/* + * 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.execution.datasources.xml + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.util.FailFastMode +import org.apache.spark.sql.execution.datasources.xml.util.XmlFile +import org.apache.spark.sql.types.StructType + +/** + * A collection of static functions for working with XML files in Spark SQL + */ +class XmlReader(private var schema: StructType, + private val options: Map[String, Any]) extends Serializable { + + private val parameters = collection.mutable.Map.empty[String, String] + options.foreach { case (k, v) => parameters(k) = v.toString } + + // Explicit constructors for Java compatibility + + def this() = { + this(null, Map.empty) + } + + def this(schema: StructType) = { + this(schema, Map.empty) + } + + def this(options: Map[String, Any]) = { + this(null, options) + } + + @deprecated("Use XmlReader(Map) with key 'charset' to specify options", "0.13.0") + def withCharset(charset: String): XmlReader = { + parameters += ("charset" -> charset) + this + } + + @deprecated("Use XmlReader(Map) with key 'codec' to specify options", "0.13.0") + def withCompression(codec: String): XmlReader = { + parameters += ("codec" -> codec) + this + } + + @deprecated("Use XmlReader(Map) with key 'rowTag' to specify options", "0.13.0") + def withRowTag(rowTag: String): XmlReader = { + parameters += ("rowTag" -> rowTag) + this + } + + @deprecated("Use XmlReader(Map) with key 'samplingRatio' to specify options", "0.13.0") + def withSamplingRatio(samplingRatio: Double): XmlReader = { + parameters += ("samplingRatio" -> samplingRatio.toString) + this + } + + @deprecated("Use XmlReader(Map) with key 'excludeAttribute' to specify options", "0.13.0") + def withExcludeAttribute(exclude: Boolean): XmlReader = { + parameters += ("excludeAttribute" -> exclude.toString) + this + } + + @deprecated("Use XmlReader(Map) with key 'treatEmptyValuesAsNulls' to specify options", "0.13.0") + def withTreatEmptyValuesAsNulls(treatAsNull: Boolean): XmlReader = { + parameters += ("treatEmptyValuesAsNulls" -> treatAsNull.toString) + this + } + + @deprecated("Use XmlReader(Map) with key 'mode' as 'FAILFAST' to specify options", "0.10.0") + def withFailFast(failFast: Boolean): XmlReader = { + if (failFast) { + parameters += ("mode" -> FailFastMode.name) + } else { + parameters -= "mode" + } + this + } + + @deprecated("Use XmlReader(Map) with key 'mode' to specify options", "0.13.0") + def withParseMode(mode: String): XmlReader = { + parameters += ("mode" -> mode) + this + } + + @deprecated("Use XmlReader(Map) with key 'attributePrefix' to specify options", "0.13.0") + def withAttributePrefix(attributePrefix: String): XmlReader = { + parameters += ("attributePrefix" -> attributePrefix) + this + } + + @deprecated("Use XmlReader(Map) with key 'valueTag' to specify options", "0.13.0") + def withValueTag(valueTag: String): XmlReader = { + parameters += ("valueTag" -> valueTag) + this + } + + @deprecated("Use XmlReader(Map) with key 'columnNameOfCorruptRecord' to specify options", + "0.13.0") + def withColumnNameOfCorruptRecord(name: String): XmlReader = { + parameters += ("columnNameOfCorruptRecord" -> name) + this + } + + @deprecated("Use XmlReader(Map) with key 'ignoreSurroundingSpaces' to specify options", "0.13.0") + def withIgnoreSurroundingSpaces(ignore: Boolean): XmlReader = { + parameters += ("ignoreSurroundingSpaces" -> ignore.toString) + this + } + + @deprecated("Use XmlReader(StructType) to specify schema", "0.13.0") + def withSchema(schema: StructType): XmlReader = { + this.schema = schema + this + } + + @deprecated("Use XmlReader(Map) with key 'rowValidationXSDPath' to specify options", "0.13.0") + def withRowValidationXSDPath(path: String): XmlReader = { + parameters += ("rowValidationXSDPath" -> path) + this + } + + /** + * @param spark current SparkSession + * @param path path to XML files to parse + * @return XML parsed as a DataFrame + */ + def xmlFile(spark: SparkSession, path: String): DataFrame = { + // We need the `charset` and `rowTag` before creating the relation. + val (charset, rowTag) = { + val options = XmlOptions(parameters.toMap) + (options.charset, options.rowTag) + } + val relation = XmlRelation( + () => XmlFile.withCharset(spark.sparkContext, path, charset, rowTag), + Some(path), + parameters.toMap, + schema)(spark.sqlContext) + spark.baseRelationToDataFrame(relation) + } + + /** + * @param spark current SparkSession + * @param ds XML for individual 'rows' as Strings + * @return XML parsed as a DataFrame + */ + def xmlDataset(spark: SparkSession, ds: Dataset[String]): DataFrame = { + xmlRdd(spark, ds.rdd) + } + + /** + * @param spark current SparkSession + * @param xmlRDD XML for individual 'rows' as Strings + * @return XML parsed as a DataFrame + */ + def xmlRdd(spark: SparkSession, xmlRDD: RDD[String]): DataFrame = { + val relation = XmlRelation( + () => xmlRDD, + None, + parameters.toMap, + schema)(spark.sqlContext) + spark.baseRelationToDataFrame(relation) + } + + /** Returns a Schema RDD for the given XML path. */ + @deprecated("Use xmlFile(SparkSession, ...)", "0.5.0") + def xmlFile(sqlContext: SQLContext, path: String): DataFrame = { + // We need the `charset` and `rowTag` before creating the relation. + val (charset, rowTag) = { + val options = XmlOptions(parameters.toMap) + (options.charset, options.rowTag) + } + val relation = XmlRelation( + () => XmlFile.withCharset(sqlContext.sparkContext, path, charset, rowTag), + Some(path), + parameters.toMap, + schema)(sqlContext) + sqlContext.baseRelationToDataFrame(relation) + } + + @deprecated("Use xmlRdd(SparkSession, ...)", "0.5.0") + def xmlRdd(sqlContext: SQLContext, xmlRDD: RDD[String]): DataFrame = { + val relation = XmlRelation( + () => xmlRDD, + None, + parameters.toMap, + schema)(sqlContext) + sqlContext.baseRelationToDataFrame(relation) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlRelation.scala new file mode 100644 index 0000000000000..18bdb87fa617f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlRelation.scala @@ -0,0 +1,86 @@ +/* + * 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.execution.datasources.xml + +import java.io.IOException + +import org.apache.hadoop.fs.Path + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlParser +import org.apache.spark.sql.execution.datasources.xml.util.{InferSchema, XmlFile} +import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation, PrunedScan} +import org.apache.spark.sql.types._ + +case class XmlRelation protected[spark] ( + baseRDD: () => RDD[String], + location: Option[String], + parameters: Map[String, String], + userSchema: StructType = null)(@transient val sqlContext: SQLContext) + extends BaseRelation + with InsertableRelation + with PrunedScan { + + // Hacky: ensure RDD's underlying data actually already exists early on + baseRDD().partitions + + private val options = XmlOptions(parameters) + + override val schema: StructType = { + Option(userSchema).getOrElse { + InferSchema.infer( + baseRDD(), + options) + } + } + + override def buildScan(requiredColumns: Array[String]): RDD[Row] = { + val requiredFields = requiredColumns.map(schema(_)) + val requestedSchema = StructType(requiredFields) + StaxXmlParser.parse( + baseRDD(), + requestedSchema, + options) + } + + // The function below was borrowed from JSONRelation + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + val filesystemPath = location match { + case Some(p) => new Path(p) + case None => + throw new IOException(s"Cannot INSERT into table with no path defined") + } + + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + + if (overwrite) { + try { + fs.delete(filesystemPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${filesystemPath.toString} prior" + + s" to INSERT OVERWRITE a XML table:\n${e.toString}") + } + // Write the data. We assume that schema isn't changed, and we won't update it. + XmlFile.saveAsXmlFile(data, filesystemPath.toString, parameters) + } else { + throw new IllegalArgumentException("XML tables only support INSERT OVERWRITE for now.") + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/functions.scala new file mode 100644 index 0000000000000..1a22652ac4542 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/functions.scala @@ -0,0 +1,42 @@ +/* + * 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.execution.datasources.xml + +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.types.DataType + +/** + * Support functions for working with XML columns directly. + */ +// scalastyle:off: object.name +object functions { +// scalastyle:on: object.name + /** + * Parses a column containing a XML string into a `StructType` with the specified schema. + * + * @param e a string column containing XML data + * @param schema the schema to use when parsing the XML string. Must be a StructType if + * column is string-valued, or ArrayType[StructType] if column is an array of strings + * @param options key-value pairs that correspond to those supported by [[XmlOptions]] + */ + def from_xml(e: Column, schema: DataType, options: Map[String, String] = Map.empty): Column = { + val expr = CatalystSqlParser.parseExpression(e.toString()) + new Column(XmlDataToCatalyst(expr, schema, XmlOptions(options))) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/package.scala new file mode 100644 index 0000000000000..7531f1f6f4843 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/package.scala @@ -0,0 +1,162 @@ +/* + * 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.execution.datasources + +import org.apache.hadoop.io.compress.CompressionCodec + +import org.apache.spark.sql.{DataFrame, DataFrameReader, DataFrameWriter, Dataset} +import org.apache.spark.sql.{Encoders, Row, SparkSession, SQLContext} +import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlParser +import org.apache.spark.sql.execution.datasources.xml.util.{InferSchema, XmlFile} +import org.apache.spark.sql.types.{ArrayType, StructType} + +package object xml { + /** + * Adds a method, `xmlFile`, to [[SQLContext]] that allows reading XML data. + */ + implicit class XmlContext(sqlContext: SQLContext) extends Serializable { + @deprecated("Use read.format(\"xml\") or read.xml", "0.4.0") + def xmlFile( + filePath: String, + rowTag: String = XmlOptions.DEFAULT_ROW_TAG, + samplingRatio: Double = 1.0, + excludeAttribute: Boolean = false, + treatEmptyValuesAsNulls: Boolean = false, + failFast: Boolean = false, + attributePrefix: String = XmlOptions.DEFAULT_ATTRIBUTE_PREFIX, + valueTag: String = XmlOptions.DEFAULT_VALUE_TAG, + charset: String = XmlOptions.DEFAULT_CHARSET): DataFrame = { + + val parameters = Map( + "rowTag" -> rowTag, + "samplingRatio" -> samplingRatio.toString, + "excludeAttribute" -> excludeAttribute.toString, + "treatEmptyValuesAsNulls" -> treatEmptyValuesAsNulls.toString, + "failFast" -> failFast.toString, + "attributePrefix" -> attributePrefix, + "valueTag" -> valueTag, + "charset" -> charset) + val xmlRelation = XmlRelation( + () => XmlFile.withCharset(sqlContext.sparkContext, filePath, charset, rowTag), + location = Some(filePath), + parameters = parameters)(sqlContext) + sqlContext.baseRelationToDataFrame(xmlRelation) + } + } + + /** + * Adds a method, `saveAsXmlFile`, to [[DataFrame]] that allows writing XML data. + * If compressionCodec is not null the resulting output will be compressed. + * Note that a codec entry in the parameters map will be ignored. + */ + implicit class XmlSchemaRDD(dataFrame: DataFrame) { + @deprecated("Use write.format(\"xml\") or write.xml", "0.4.0") + def saveAsXmlFile( + path: String, parameters: scala.collection.Map[String, String] = Map(), + compressionCodec: Class[_ <: CompressionCodec] = null): Unit = { + val mutableParams = collection.mutable.Map(parameters.toSeq: _*) + val safeCodec = mutableParams.get("codec") + .orElse(Option(compressionCodec).map(_.getCanonicalName)) + .orNull + mutableParams.put("codec", safeCodec) + XmlFile.saveAsXmlFile(dataFrame, path, mutableParams.toMap) + } + } + + /** + * Adds a method, `xml`, to DataFrameReader that allows you to read XML files using + * the DataFileReader + */ + implicit class XmlDataFrameReader(reader: DataFrameReader) { + def xml: String => DataFrame = reader + .format("org.apache.spark.sql.execution.datasources.xml").load + + @deprecated("Use XmlReader directly", "0.13.0") + def xml(xmlDataset: Dataset[String]): DataFrame = { + val spark = SparkSession.builder().getOrCreate() + new XmlReader().xmlDataset(spark, xmlDataset) + } + } + + /** + * Adds a method, `xml`, to DataFrameWriter that allows you to write XML files using + * the DataFileWriter + */ + implicit class XmlDataFrameWriter[T](writer: DataFrameWriter[T]) { + // Note that writing a XML file from [[DataFrame]] having a field [[ArrayType]] with + // its element as [[ArrayType]] would have an additional nested field for the element. + // For example, the [[DataFrame]] having a field below, + // + // fieldA [[data1, data2]] + // + // would produce a XML file below. + // + // + // data1 + // + // + // data2 + // + // + // Namely, roundtrip in writing and reading can end up in different schema structure. + def xml: String => Unit = writer + .format("org.apache.spark.sql.execution.datasources.xml").save + } + + /** + * Infers the schema of XML documents as strings. + * + * @param ds Dataset of XML strings + * @param options additional XML parsing options + * @return inferred schema for XML + */ + def schema_of_xml(ds: Dataset[String], options: Map[String, String] = Map.empty): StructType = + InferSchema.infer(ds.rdd, XmlOptions(options)) + + /** + * Infers the schema of XML documents as strings. + * + * @param df one-column DataFrame of XML strings + * @param options additional XML parsing options + * @return inferred schema for XML + */ + def schema_of_xml_df(df: DataFrame, options: Map[String, String] = Map.empty): StructType = + schema_of_xml(df.as[String](Encoders.STRING), options) + + /** + * Infers the schema of XML documents when inputs are arrays of strings, each an XML doc. + * + * @param ds Dataset of XML strings + * @param options additional XML parsing options + * @return inferred schema for XML. Will be an ArrayType[StructType]. + */ + def schema_of_xml_array(ds: Dataset[Array[String]], + options: Map[String, String] = Map.empty): ArrayType = + ArrayType(InferSchema.infer(ds.rdd.flatMap(a => a), XmlOptions(options))) + + /** + * @param xml XML document to parse, as string + * @param schema the schema to use when parsing the XML string + * @param options key-value pairs that correspond to those supported by [[XmlOptions]] + * @return [[Row]] representing the parsed XML structure + */ + def from_xml_string(xml: String, schema: StructType, + options: Map[String, String] = Map.empty): Row = { + StaxXmlParser.parseColumn(xml, schema, XmlOptions(options)) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala new file mode 100644 index 0000000000000..eb4c94fe2802b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala @@ -0,0 +1,159 @@ +/* + * 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.execution.datasources.xml.parsers + +import java.sql.{Date, Timestamp} +import java.time.format.DateTimeFormatter +import javax.xml.stream.XMLStreamWriter + +import scala.collection.Map + +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.types._ + +// This class is borrowed from Spark json datasource. +private[xml] object StaxXmlGenerator { + + /** + * Transforms a single Row to XML + * + * @param schema the schema object used for conversion + * @param writer a XML writer object + * @param options options for XML datasource. + * @param row The row to convert + */ + def apply( + schema: StructType, + writer: XMLStreamWriter, + options: XmlOptions)(row: Row): Unit = { + + require(options.attributePrefix.nonEmpty, + "'attributePrefix' option should not be empty string.") + + def writeChildElement(name: String, dt: DataType, v: Any): Unit = (name, dt, v) match { + // If this is meant to be value but in no child, write only a value + case (_, _, null) | (_, NullType, _) if options.nullValue == null => + // Because usually elements having `null` do not exist, just do not write + // elements when given values are `null`. + case (_, _, _) if name == options.valueTag => + // If this is meant to be value but in no child, write only a value + writeElement(dt, v, options) + case (_, _, _) => + writer.writeStartElement(name) + writeElement(dt, v, options) + writer.writeEndElement() + } + + def writeChild(name: String, dt: DataType, v: Any): Unit = { + (dt, v) match { + // If this is meant to be attribute, write an attribute + case (_, null) | (NullType, _) + if name.startsWith(options.attributePrefix) && name != options.valueTag => + Option(options.nullValue).foreach { + writer.writeAttribute(name.substring(options.attributePrefix.length), _) + } + case _ if name.startsWith(options.attributePrefix) && name != options.valueTag => + writer.writeAttribute(name.substring(options.attributePrefix.length), v.toString) + + // For ArrayType, we just need to write each as XML element. + case (ArrayType(ty, _), v: scala.collection.Seq[_]) => + v.foreach { e => + writeChildElement(name, ty, e) + } + // For other datatypes, we just write normal elements. + case _ => + writeChildElement(name, dt, v) + } + } + + def writeElement(dt: DataType, v: Any, options: XmlOptions): Unit = (dt, v) match { + case (_, null) | (NullType, _) => writer.writeCharacters(options.nullValue) + case (StringType, v: String) => writer.writeCharacters(v) + case (TimestampType, v: Timestamp) => + val formatter = options.timestampFormat.map(DateTimeFormatter.ofPattern). + getOrElse(DateTimeFormatter.ISO_INSTANT) + writer.writeCharacters(formatter.format(v.toInstant())) + case (DateType, v: Date) => + val formatter = options.dateFormat.map(DateTimeFormatter.ofPattern). + getOrElse(DateTimeFormatter.ISO_DATE) + writer.writeCharacters(formatter.format(v.toLocalDate())) + case (IntegerType, v: Int) => writer.writeCharacters(v.toString) + case (ShortType, v: Short) => writer.writeCharacters(v.toString) + case (FloatType, v: Float) => writer.writeCharacters(v.toString) + case (DoubleType, v: Double) => writer.writeCharacters(v.toString) + case (LongType, v: Long) => writer.writeCharacters(v.toString) + case (DecimalType(), v: java.math.BigDecimal) => writer.writeCharacters(v.toString) + case (ByteType, v: Byte) => writer.writeCharacters(v.toString) + case (BooleanType, v: Boolean) => writer.writeCharacters(v.toString) + + // For the case roundtrip in reading and writing XML files, [[ArrayType]] cannot have + // [[ArrayType]] as element type. It always wraps the element with [[StructType]]. So, + // this case only can happen when we convert a normal [[DataFrame]] to XML file. + // When [[ArrayType]] has [[ArrayType]] as elements, it is confusing what is element name + // for XML file. + case (ArrayType(ty, _), v: scala.collection.Seq[_]) => + v.foreach { e => + writeChild(options.arrayElementName, ty, e) + } + + case (MapType(_, vt, _), mv: Map[_, _]) => + val (attributes, elements) = mv.toSeq.partition { case (f, _) => + f.toString.startsWith(options.attributePrefix) && f.toString != options.valueTag + } + // We need to write attributes first before the value. + (attributes ++ elements).foreach { + case (k, v) => + writeChild(k.toString, vt, v) + } + + case (StructType(ty), r: Row) => + val (attributes, elements) = ty.zip(r.toSeq).partition { case (f, _) => + f.name.startsWith(options.attributePrefix) && f.name != options.valueTag + } + // We need to write attributes first before the value. + (attributes ++ elements).foreach { + case (field, value) => + writeChild(field.name, field.dataType, value) + } + + case (_, _) => + throw new IllegalArgumentException( + s"Failed to convert value $v (class of ${v.getClass}) in type $dt to XML.") + } + + val (attributes, elements) = schema.zip(row.toSeq).partition { case (f, _) => + f.name.startsWith(options.attributePrefix) && f.name != options.valueTag + } + // Writing attributes + writer.writeStartElement(options.rowTag) + attributes.foreach { + case (f, v) if v == null || f.dataType == NullType => + Option(options.nullValue).foreach { + writer.writeAttribute(f.name.substring(options.attributePrefix.length), _) + } + case (f, v) => + writer.writeAttribute(f.name.substring(options.attributePrefix.length), v.toString) + } + // Writing elements + val (names, values) = elements.unzip + val elementSchema = StructType(schema.filter(names.contains)) + val elementRow = Row.fromSeq(row.toSeq.filter(values.contains)) + writeElement(elementSchema, elementRow, options) + writer.writeEndElement() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParser.scala new file mode 100644 index 0000000000000..89b140fe28e59 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParser.scala @@ -0,0 +1,375 @@ +/* + * 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.execution.datasources.xml.parsers + +import java.io.StringReader +import javax.xml.stream.XMLEventReader +import javax.xml.stream.events.{Attribute, Characters, EndElement, StartElement, XMLEvent} +import javax.xml.transform.stream.StreamSource +import javax.xml.validation.Schema + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.Try +import scala.util.control.NonFatal + +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.execution.datasources.xml.util._ +import org.apache.spark.sql.execution.datasources.xml.util.TypeCast._ +import org.apache.spark.sql.types._ + +/** + * Wraps parser to iteration process. + */ +private[xml] object StaxXmlParser extends Serializable with Logging { + def parse( + xml: RDD[String], + schema: StructType, + options: XmlOptions): RDD[Row] = { + xml.mapPartitions { iter => + val xsdSchema = Option(options.rowValidationXSDPath).map(ValidatorUtil.getSchema) + iter.flatMap { xml => + doParseColumn(xml, schema, options, options.parseMode, xsdSchema) + } + } + } + + def parseColumn(xml: String, schema: StructType, options: XmlOptions): Row = { + // The user=specified schema from from_xml, etc will typically not include a + // "corrupted record" column. In PERMISSIVE mode, which puts bad records in + // such a column, this would cause an error. In this mode, if such a column + // is not manually specified, then fall back to DROPMALFORMED, which will return + // null column values where parsing fails. + val parseMode = + if (options.parseMode == PermissiveMode && + !schema.fields.exists(_.name == options.columnNameOfCorruptRecord)) { + DropMalformedMode + } else { + options.parseMode + } + val xsdSchema = Option(options.rowValidationXSDPath).map(ValidatorUtil.getSchema) + doParseColumn(xml, schema, options, parseMode, xsdSchema).orNull + } + + private def doParseColumn(xml: String, + schema: StructType, + options: XmlOptions, + parseMode: ParseMode, + xsdSchema: Option[Schema]): Option[Row] = { + try { + xsdSchema.foreach { schema => + schema.newValidator().validate(new StreamSource(new StringReader(xml))) + } + val parser = StaxXmlParserUtils.filteredReader(xml) + val rootAttributes = StaxXmlParserUtils.gatherRootAttributes(parser) + Some(convertObject(parser, schema, options, rootAttributes)) + } catch { + case e: PartialResultException => + failedRecord(xml, options, parseMode, schema, + e.cause, Some(e.partialResult)) + case NonFatal(e) => + failedRecord(xml, options, parseMode, schema, e) + } + } + + private def failedRecord(record: String, + options: XmlOptions, + parseMode: ParseMode, + schema: StructType, + cause: Throwable = null, + partialResult: Option[Row] = None): Option[Row] = { + // create a row even if no corrupt record column is present + val abbreviatedRecord = + (if (record.length() > 1000) record.substring(0, 1000) + "..." else record). + replaceAll("\n", "") + parseMode match { + case FailFastMode => + logInfo(s"Malformed line: $abbreviatedRecord") + logDebug("Caused by:", cause) + throw new IllegalArgumentException("Malformed line in FAILFAST mode", cause) + case DropMalformedMode => + logInfo(s"Malformed line: $abbreviatedRecord") + logDebug("Caused by:", cause) + None + case PermissiveMode => + logDebug(s"Malformed line: $abbreviatedRecord") + logDebug("Caused by:", cause) + // The logic below is borrowed from Apache Spark's FailureSafeParser. + val resultRow = new Array[Any](schema.length) + schema.filterNot(_.name == options.columnNameOfCorruptRecord).foreach { from => + val sourceIndex = schema.fieldIndex(from.name) + resultRow(sourceIndex) = partialResult.map(_.get(sourceIndex)).orNull + } + val corruptFieldIndex = Try(schema.fieldIndex(options.columnNameOfCorruptRecord)).toOption + corruptFieldIndex.foreach(resultRow(_) = record) + Some(Row.fromSeq(resultRow.toIndexedSeq)) + } + } + + /** + * Parse the current token (and related children) according to a desired schema + */ + private[xml] def convertField( + parser: XMLEventReader, + dataType: DataType, + options: XmlOptions, + attributes: Array[Attribute] = Array.empty): Any = { + + def convertComplicatedType(dt: DataType, attributes: Array[Attribute]): Any = dt match { + case st: StructType => convertObject(parser, st, options) + case MapType(StringType, vt, _) => convertMap(parser, vt, options, attributes) + case ArrayType(st, _) => convertField(parser, st, options) + case _: StringType => + convertTo(StaxXmlParserUtils.currentStructureAsString(parser), StringType, options) + } + + (parser.peek, dataType) match { + case (_: StartElement, dt: DataType) => convertComplicatedType(dt, attributes) + case (_: EndElement, _: StringType) => + // Empty. It's null if these are explicitly treated as null, or "" is the null value + if (options.treatEmptyValuesAsNulls || options.nullValue == "") { + null + } else { + "" + } + case (_: EndElement, _: DataType) => null + case (c: Characters, ArrayType(st, _)) => + // For `ArrayType`, it needs to return the type of element. The values are merged later. + convertTo(c.getData, st, options) + case (c: Characters, st: StructType) => + // If a value tag is present, this can be an attribute-only element whose values is in that + // value tag field. Or, it can be a mixed-type element with both some character elements + // and other complex structure. Character elements are ignored. + val attributesOnly = st.fields.forall { f => + f.name == options.valueTag || f.name.startsWith(options.attributePrefix) + } + if (attributesOnly) { + // If everything else is an attribute column, there's no complex structure. + // Just return the value of the character element, or null if we don't have a value tag + st.find(_.name == options.valueTag).map( + valueTag => convertTo(c.getData, valueTag.dataType, options)).orNull + } else { + // Otherwise, ignore this character element, and continue parsing the following complex + // structure + parser.next + parser.peek match { + case _: EndElement => null // no struct here at all; done + case _ => convertObject(parser, st, options) + } + } + case (_: Characters, _: StringType) => + convertTo(StaxXmlParserUtils.currentStructureAsString(parser), StringType, options) + case (c: Characters, _: DataType) if c.isWhiteSpace => + // When `Characters` is found, we need to look further to decide + // if this is really data or space between other elements. + val data = c.getData + parser.next + parser.peek match { + case _: StartElement => convertComplicatedType(dataType, attributes) + case _: EndElement if data.isEmpty => null + case _: EndElement if options.treatEmptyValuesAsNulls => null + case _: EndElement => convertTo(data, dataType, options) + case _ => convertField(parser, dataType, options, attributes) + } + case (c: Characters, dt: DataType) => + convertTo(c.getData, dt, options) + case (e: XMLEvent, dt: DataType) => + throw new IllegalArgumentException( + s"Failed to parse a value for data type $dt with event ${e.toString}") + } + } + + /** + * Parse an object as map. + */ + private def convertMap( + parser: XMLEventReader, + valueType: DataType, + options: XmlOptions, + attributes: Array[Attribute]): Map[String, Any] = { + val kvPairs = ArrayBuffer.empty[(String, Any)] + attributes.foreach { attr => + kvPairs += (options.attributePrefix + attr.getName.getLocalPart -> attr.getValue) + } + var shouldStop = false + while (!shouldStop) { + parser.nextEvent match { + case e: StartElement => + kvPairs += + (StaxXmlParserUtils.getName(e.asStartElement.getName, options) -> + convertField(parser, valueType, options)) + case _: EndElement => + shouldStop = StaxXmlParserUtils.checkEndElement(parser) + case _ => // do nothing + } + } + kvPairs.toMap + } + + /** + * Convert XML attributes to a map with the given schema types. + */ + private def convertAttributes( + attributes: Array[Attribute], + schema: StructType, + options: XmlOptions): Map[String, Any] = { + val convertedValuesMap = collection.mutable.Map.empty[String, Any] + val valuesMap = StaxXmlParserUtils.convertAttributesToValuesMap(attributes, options) + valuesMap.foreach { case (f, v) => + val nameToIndex = schema.map(_.name).zipWithIndex.toMap + nameToIndex.get(f).foreach { i => + convertedValuesMap(f) = convertTo(v, schema(i).dataType, options) + } + } + convertedValuesMap.toMap + } + + /** + * [[convertObject()]] calls this in order to convert the nested object to a row. + * [[convertObject()]] contains some logic to find out which events are the start + * and end of a nested row and this function converts the events to a row. + */ + private def convertObjectWithAttributes( + parser: XMLEventReader, + schema: StructType, + options: XmlOptions, + attributes: Array[Attribute] = Array.empty): Row = { + // TODO: This method might have to be removed. Some logics duplicate `convertObject()` + val row = new Array[Any](schema.length) + + // Read attributes first. + val attributesMap = convertAttributes(attributes, schema, options) + + // Then, we read elements here. + val fieldsMap = convertField(parser, schema, options) match { + case row: Row => + Map(schema.map(_.name).zip(row.toSeq): _*) + case v if schema.fieldNames.contains(options.valueTag) => + // If this is the element having no children, then it wraps attributes + // with a row So, we first need to find the field name that has the real + // value and then push the value. + val valuesMap = schema.fieldNames.map((_, null)).toMap + valuesMap + (options.valueTag -> v) + case _ => Map.empty + } + + // Here we merge both to a row. + val valuesMap = fieldsMap ++ attributesMap + valuesMap.foreach { case (f, v) => + val nameToIndex = schema.map(_.name).zipWithIndex.toMap + nameToIndex.get(f).foreach { row(_) = v } + } + + if (valuesMap.isEmpty) { + // Return an empty row with all nested elements by the schema set to null. + Row.fromSeq(Seq.fill(schema.fieldNames.length)(null)) + } else { + Row.fromSeq(row.toIndexedSeq) + } + } + + /** + * Parse an object from the event stream into a new Row representing the schema. + * Fields in the xml that are not defined in the requested schema will be dropped. + */ + private def convertObject( + parser: XMLEventReader, + schema: StructType, + options: XmlOptions, + rootAttributes: Array[Attribute] = Array.empty): Row = { + val row = new Array[Any](schema.length) + val nameToIndex = schema.map(_.name).zipWithIndex.toMap + // If there are attributes, then we process them first. + convertAttributes(rootAttributes, schema, options).toSeq.foreach { case (f, v) => + nameToIndex.get(f).foreach { row(_) = v } + } + + val wildcardColName = options.wildcardColName + val hasWildcard = schema.exists(_.name == wildcardColName) + + var badRecordException: Option[Throwable] = None + + var shouldStop = false + while (!shouldStop) { + parser.nextEvent match { + case e: StartElement => try { + val attributes = e.getAttributes.asScala.map(_.asInstanceOf[Attribute]).toArray + val field = StaxXmlParserUtils.getName(e.asStartElement.getName, options) + + nameToIndex.get(field) match { + case Some(index) => schema(index).dataType match { + case st: StructType => + row(index) = convertObjectWithAttributes(parser, st, options, attributes) + + case ArrayType(dt: DataType, _) => + val values = Option(row(index)) + .map(_.asInstanceOf[ArrayBuffer[Any]]) + .getOrElse(ArrayBuffer.empty[Any]) + val newValue = dt match { + case st: StructType => + convertObjectWithAttributes(parser, st, options, attributes) + case dt: DataType => + convertField(parser, dt, options) + } + row(index) = values :+ newValue + + case dt: DataType => + row(index) = convertField(parser, dt, options, attributes) + } + + case None => + if (hasWildcard) { + // Special case: there's an 'any' wildcard element that matches anything else + // as a string (or array of strings, to parse multiple ones) + val newValue = convertField(parser, StringType, options) + val anyIndex = schema.fieldIndex(wildcardColName) + schema(wildcardColName).dataType match { + case StringType => + row(anyIndex) = newValue + case ArrayType(StringType, _) => + val values = Option(row(anyIndex)) + .map(_.asInstanceOf[ArrayBuffer[String]]) + .getOrElse(ArrayBuffer.empty[String]) + row(anyIndex) = values :+ newValue + } + } else { + StaxXmlParserUtils.skipChildren(parser) + } + } + } catch { + case NonFatal(exception) if options.parseMode == PermissiveMode => + badRecordException = badRecordException.orElse(Some(exception)) + } + + case _: EndElement => + shouldStop = StaxXmlParserUtils.checkEndElement(parser) + + case _ => // do nothing + } + } + + if (badRecordException.isEmpty) { + Row.fromSeq(row.toIndexedSeq) + } else { + throw PartialResultException(Row.fromSeq(row.toIndexedSeq), badRecordException.get) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtils.scala new file mode 100644 index 0000000000000..c7f31b338398a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtils.scala @@ -0,0 +1,179 @@ +/* + * 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.execution.datasources.xml.parsers + +import java.io.StringReader +import javax.xml.namespace.QName +import javax.xml.stream.{EventFilter, XMLEventReader, XMLInputFactory, XMLStreamConstants} +import javax.xml.stream.events._ + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.execution.datasources.xml.XmlOptions + +private[xml] object StaxXmlParserUtils { + + private[xml] val factory: XMLInputFactory = { + val factory = XMLInputFactory.newInstance() + factory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false) + factory.setProperty(XMLInputFactory.IS_COALESCING, true) + factory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false) + factory.setProperty(XMLInputFactory.SUPPORT_DTD, false) + factory + } + + def filteredReader(xml: String): XMLEventReader = { + val filter = new EventFilter { + override def accept(event: XMLEvent): Boolean = + // Ignore comments and processing instructions + event.getEventType match { + case XMLStreamConstants.COMMENT | XMLStreamConstants.PROCESSING_INSTRUCTION => false + case _ => true + } + } + // It does not have to skip for white space, since `XmlInputFormat` + // always finds the root tag without a heading space. + val eventReader = factory.createXMLEventReader(new StringReader(xml)) + factory.createFilteredReader(eventReader, filter) + } + + def gatherRootAttributes(parser: XMLEventReader): Array[Attribute] = { + val rootEvent = + StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.START_ELEMENT) + rootEvent.asStartElement.getAttributes.asScala.map(_.asInstanceOf[Attribute]).toArray + } + + /** + * Skips elements until this meets the given type of a element + */ + def skipUntil(parser: XMLEventReader, eventType: Int): XMLEvent = { + var event = parser.peek + while (parser.hasNext && event.getEventType != eventType) { + event = parser.nextEvent + } + event + } + + /** + * Checks if current event points the EndElement. + */ + @tailrec + def checkEndElement(parser: XMLEventReader): Boolean = { + parser.peek match { + case _: EndElement | _: EndDocument => true + case _: StartElement => false + case _ => + // When other events are found here rather than `EndElement` or `StartElement` + // , we need to look further to decide if this is the end because this can be + // whitespace between `EndElement` and `StartElement`. + parser.nextEvent + checkEndElement(parser) + } + } + + /** + * Produces values map from given attributes. + */ + def convertAttributesToValuesMap( + attributes: Array[Attribute], + options: XmlOptions): Map[String, String] = { + if (options.excludeAttributeFlag) { + Map.empty[String, String] + } else { + attributes.map { attr => + val key = options.attributePrefix + getName(attr.getName, options) + val value = attr.getValue match { + case v if options.treatEmptyValuesAsNulls && v.trim.isEmpty => null + case v => v + } + key -> value + }.toMap + } + } + + /** + * Gets the local part of an XML name, optionally without namespace. + */ + def getName(name: QName, options: XmlOptions): String = { + val localPart = name.getLocalPart + // Ignore namespace prefix up to last : if configured + if (options.ignoreNamespace) { + localPart.split(":").last + } else { + localPart + } + } + + /** + * Convert the current structure of XML document to a XML string. + */ + def currentStructureAsString(parser: XMLEventReader): String = { + val xmlString = new StringBuilder() + var indent = 0 + do { + parser.nextEvent match { + case e: StartElement => + xmlString.append('<').append(e.getName) + e.getAttributes.asScala.foreach { a => + val att = a.asInstanceOf[Attribute] + xmlString.append(' ').append(att.getName).append("=\""). + append(att.getValue).append('"') + } + xmlString.append('>') + indent += 1 + case e: EndElement => + xmlString.append("') + indent -= 1 + case c: Characters => + xmlString.append(c.getData) + case _: XMLEvent => // do nothing + } + } while (parser.peek() match { + case _: EndElement => + // until the unclosed end element for the whole parent is found + indent > 0 + case _ => true + }) + xmlString.toString() + } + + /** + * Skip the children of the current XML element. + */ + def skipChildren(parser: XMLEventReader): Unit = { + var shouldStop = checkEndElement(parser) + while (!shouldStop) { + parser.nextEvent match { + case _: StartElement => + val e = parser.peek + if (e.isCharacters && e.asCharacters.isWhiteSpace) { + // There can be a `Characters` event between `StartElement`s. + // So, we need to check further to decide if this is a data or just + // a whitespace between them. + parser.next + } + if (parser.peek.isStartElement) { + skipChildren(parser) + } + case _: EndElement => + shouldStop = checkEndElement(parser) + case _: XMLEvent => // do nothing + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/InferSchema.scala new file mode 100644 index 0000000000000..4f540d12752e3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/InferSchema.scala @@ -0,0 +1,336 @@ +/* + * 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.execution.datasources.xml.util + +import java.io.StringReader +import javax.xml.stream.XMLEventReader +import javax.xml.stream.events.{Attribute, Characters, EndElement, StartElement, XMLEvent} +import javax.xml.transform.stream.StreamSource + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.util.PermissiveMode +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlParserUtils +import org.apache.spark.sql.execution.datasources.xml.util.TypeCast._ +import org.apache.spark.sql.types._ + +private[xml] object InferSchema { + + /** + * Copied from internal Spark api + * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]] + */ + private val numericPrecedence: IndexedSeq[DataType] = + IndexedSeq[DataType]( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + TimestampType, + DecimalType.SYSTEM_DEFAULT) + + private val findTightestCommonTypeOfTwo: (DataType, DataType) => Option[DataType] = { + case (t1, t2) if t1 == t2 => Some(t1) + + // Promote numeric types to the highest of the two + case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => + val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) + Some(numericPrecedence(index)) + + case _ => None + } + + /** + * Infer the type of a collection of XML records in three stages: + * 1. Infer the type of each record + * 2. Merge types by choosing the lowest type necessary to cover equal keys + * 3. Replace any remaining null fields with string, the top type + */ + def infer(xml: RDD[String], options: XmlOptions): StructType = { + val schemaData = if (options.samplingRatio < 1.0) { + xml.sample(withReplacement = false, options.samplingRatio, 1) + } else { + xml + } + // perform schema inference on each row and merge afterwards + val rootType = schemaData.mapPartitions { iter => + val xsdSchema = Option(options.rowValidationXSDPath).map(ValidatorUtil.getSchema) + + iter.flatMap { xml => + try { + xsdSchema.foreach { schema => + schema.newValidator().validate(new StreamSource(new StringReader(xml))) + } + + val parser = StaxXmlParserUtils.filteredReader(xml) + val rootAttributes = StaxXmlParserUtils.gatherRootAttributes(parser) + Some(inferObject(parser, options, rootAttributes)) + } catch { + case NonFatal(_) if options.parseMode == PermissiveMode => + Some(StructType(Seq(StructField(options.columnNameOfCorruptRecord, StringType)))) + case NonFatal(_) => + None + } + } + }.fold(StructType(Seq()))(compatibleType(options)) + + canonicalizeType(rootType) match { + case Some(st: StructType) => st + case _ => + // canonicalizeType erases all empty structs, including the only one we want to keep + StructType(Seq()) + } + } + + private def inferFrom(datum: String, options: XmlOptions): DataType = { + val value = if (datum != null && options.ignoreSurroundingSpaces) { + datum.trim() + } else { + datum + } + + if (options.inferSchema) { + value match { + case null => NullType + case v if v.isEmpty => NullType + case v if isLong(v) => LongType + case v if isInteger(v) => IntegerType + case v if isDouble(v) => DoubleType + case v if isBoolean(v) => BooleanType + case v if isTimestamp(v, options) => TimestampType + case v if isDate(v, options) => DateType + case _ => StringType + } + } else { + StringType + } + } + + @tailrec + private def inferField(parser: XMLEventReader, options: XmlOptions): DataType = { + parser.peek match { + case _: EndElement => NullType + case _: StartElement => inferObject(parser, options) + case c: Characters if c.isWhiteSpace => + // When `Characters` is found, we need to look further to decide + // if this is really data or space between other elements. + val data = c.getData + parser.nextEvent() + parser.peek match { + case _: StartElement => inferObject(parser, options) + case _: EndElement if data.isEmpty => NullType + case _: EndElement if options.treatEmptyValuesAsNulls => NullType + case _: EndElement => StringType + case _ => inferField(parser, options) + } + case c: Characters if !c.isWhiteSpace => + // This could be the characters of a character-only element, or could have mixed + // characters and other complex structure + val characterType = inferFrom(c.getData, options) + parser.nextEvent() + parser.peek match { + case _: StartElement => + // Some more elements follow; so ignore the characters. + // Use the schema of the rest + inferObject(parser, options).asInstanceOf[StructType] + case _ => + // That's all, just the character-only body; use that as the type + characterType + } + case e: XMLEvent => + throw new IllegalArgumentException(s"Failed to parse data with unexpected event $e") + } + } + + /** + * Infer the type of a xml document from the parser's token stream + */ + private def inferObject( + parser: XMLEventReader, + options: XmlOptions, + rootAttributes: Array[Attribute] = Array.empty): DataType = { + val builder = ArrayBuffer[StructField]() + val nameToDataType = collection.mutable.Map.empty[String, ArrayBuffer[DataType]] + // If there are attributes, then we should process them first. + val rootValuesMap = + StaxXmlParserUtils.convertAttributesToValuesMap(rootAttributes, options) + rootValuesMap.foreach { + case (f, v) => + nameToDataType += (f -> ArrayBuffer(inferFrom(v, options))) + } + var shouldStop = false + while (!shouldStop) { + parser.nextEvent match { + case e: StartElement => + val attributes = e.getAttributes.asScala.map(_.asInstanceOf[Attribute]).toArray + val valuesMap = StaxXmlParserUtils.convertAttributesToValuesMap(attributes, options) + val inferredType = inferField(parser, options) match { + case st: StructType if valuesMap.nonEmpty => + // Merge attributes to the field + val nestedBuilder = ArrayBuffer[StructField]() + nestedBuilder ++= st.fields + valuesMap.foreach { + case (f, v) => + nestedBuilder += StructField(f, inferFrom(v, options), nullable = true) + } + StructType(nestedBuilder.sortBy(_.name).toArray) + + case dt: DataType if valuesMap.nonEmpty => + // We need to manually add the field for value. + val nestedBuilder = ArrayBuffer[StructField]() + nestedBuilder += StructField(options.valueTag, dt, nullable = true) + valuesMap.foreach { + case (f, v) => + nestedBuilder += StructField(f, inferFrom(v, options), nullable = true) + } + StructType(nestedBuilder.sortBy(_.name).toArray) + + case dt: DataType => dt + } + // Add the field and datatypes so that we can check if this is ArrayType. + val field = StaxXmlParserUtils.getName(e.asStartElement.getName, options) + val dataTypes = nameToDataType.getOrElse(field, ArrayBuffer.empty[DataType]) + dataTypes += inferredType + nameToDataType += (field -> dataTypes) + + case _: EndElement => + shouldStop = StaxXmlParserUtils.checkEndElement(parser) + + case _ => // do nothing + } + } + // We need to manually merges the fields having the sames so that + // This can be inferred as ArrayType. + nameToDataType.foreach { + case (field, dataTypes) if dataTypes.length > 1 => + val elementType = dataTypes.reduceLeft(InferSchema.compatibleType(options)) + builder += StructField(field, ArrayType(elementType), nullable = true) + case (field, dataTypes) => + builder += StructField(field, dataTypes.head, nullable = true) + } + + // Note: other code relies on this sorting for correctness, so don't remove it! + StructType(builder.sortBy(_.name).toArray) + } + + /** + * Convert NullType to StringType and remove StructTypes with no fields + */ + private def canonicalizeType(dt: DataType): Option[DataType] = dt match { + case at @ ArrayType(elementType, _) => + for { + canonicalType <- canonicalizeType(elementType) + } yield { + at.copy(canonicalType) + } + + case StructType(fields) => + val canonicalFields = for { + field <- fields if field.name.nonEmpty + canonicalType <- canonicalizeType(field.dataType) + } yield { + field.copy(dataType = canonicalType) + } + + if (canonicalFields.nonEmpty) { + Some(StructType(canonicalFields)) + } else { + // per SPARK-8093: empty structs should be deleted + None + } + + case NullType => Some(StringType) + case other => Some(other) + } + + /** + * Returns the most general data type for two given data types. + */ + private[xml] def compatibleType(options: XmlOptions)(t1: DataType, t2: DataType): DataType = { + // TODO: Optimise this logic. + findTightestCommonTypeOfTwo(t1, t2).getOrElse { + // t1 or t2 is a StructType, ArrayType, or an unexpected type. + (t1, t2) match { + // Double support larger range than fixed decimal, DecimalType.Maximum should be enough + // in most case, also have better precision. + case (DoubleType, _: DecimalType) => + DoubleType + case (_: DecimalType, DoubleType) => + DoubleType + case (t1: DecimalType, t2: DecimalType) => + val scale = math.max(t1.scale, t2.scale) + val range = math.max(t1.precision - t1.scale, t2.precision - t2.scale) + if (range + scale > 38) { + // DecimalType can't support precision > 38 + DoubleType + } else { + DecimalType(range + scale, scale) + } + + case (StructType(fields1), StructType(fields2)) => + val newFields = (fields1 ++ fields2).groupBy(_.name).map { + case (name, fieldTypes) => + val dataType = fieldTypes.map(_.dataType).reduce(compatibleType(options)) + StructField(name, dataType, nullable = true) + } + StructType(newFields.toArray.sortBy(_.name)) + + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType( + compatibleType(options)(elementType1, elementType2), containsNull1 || containsNull2) + + // In XML datasource, since StructType can be compared with ArrayType. + // In this case, ArrayType wraps the StructType. + case (ArrayType(ty1, _), ty2) => + ArrayType(compatibleType(options)(ty1, ty2)) + + case (ty1, ArrayType(ty2, _)) => + ArrayType(compatibleType(options)(ty1, ty2)) + + // As this library can infer an element with attributes as StructType whereas + // some can be inferred as other non-structural data types, this case should be + // treated. + case (st: StructType, dt: DataType) if st.fieldNames.contains(options.valueTag) => + val valueIndex = st.fieldNames.indexOf(options.valueTag) + val valueField = st.fields(valueIndex) + val valueDataType = compatibleType(options)(valueField.dataType, dt) + st.fields(valueIndex) = StructField(options.valueTag, valueDataType, nullable = true) + st + + case (dt: DataType, st: StructType) if st.fieldNames.contains(options.valueTag) => + val valueIndex = st.fieldNames.indexOf(options.valueTag) + val valueField = st.fields(valueIndex) + val valueDataType = compatibleType(options)(dt, valueField.dataType) + st.fields(valueIndex) = StructField(options.valueTag, valueDataType, nullable = true) + st + + // TODO: These null type checks should be in `findTightestCommonTypeOfTwo`. + case (_, NullType) => t1 + case (NullType, _) => t2 + // strings and every string is a XML object. + case (_, _) => StringType + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/PartialResultException.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/PartialResultException.scala new file mode 100644 index 0000000000000..26acd814ba26c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/PartialResultException.scala @@ -0,0 +1,29 @@ +/* + * 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.execution.datasources.xml.util + +import org.apache.spark.sql.Row + +/** + * Exception thrown when the underlying parser returns a partial result of parsing. + * @param partialResult the partial result of parsing a bad record. + * @param cause the actual exception about why the parser cannot return full result. + */ +case class PartialResultException( + partialResult: Row, + cause: Throwable) + extends Exception(cause) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCast.scala new file mode 100644 index 0000000000000..c91f423a62c24 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCast.scala @@ -0,0 +1,297 @@ +/* + * 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.execution.datasources.xml.util + +import java.math.BigDecimal +import java.sql.{Date, Timestamp} +import java.text.NumberFormat +import java.time.{Instant, LocalDate, ZoneId} +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder} +import java.util.Locale + +import scala.util.Try +import scala.util.control.Exception._ + +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.types._ + +/** + * Utility functions for type casting + */ +private[xml] object TypeCast { + + /** + * Casts given string datum to specified type. + * Currently we do not support complex types (ArrayType, MapType, StructType). + * + * For string types, this is simply the datum. For other types. + * For other nullable types, this is null if the string datum is empty. + * + * @param datum string value + * @param castType SparkSQL type + */ + private[xml] def castTo( + datum: String, + castType: DataType, + options: XmlOptions): Any = { + if ((datum == options.nullValue) || + (options.treatEmptyValuesAsNulls && datum == "")) { + null + } else { + castType match { + case _: ByteType => datum.toByte + case _: ShortType => datum.toShort + case _: IntegerType => datum.toInt + case _: LongType => datum.toLong + case _: FloatType => Try(datum.toFloat) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) + case _: DoubleType => Try(datum.toDouble) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) + case _: BooleanType => parseXmlBoolean(datum) + case dt: DecimalType => + Decimal(new BigDecimal(datum.replaceAll(",", "")), dt.precision, dt.scale) + case _: TimestampType => parseXmlTimestamp(datum, options) + case _: DateType => parseXmlDate(datum, options) + case _: StringType => datum + case _ => throw new IllegalArgumentException(s"Unsupported type: ${castType.typeName}") + } + } + } + + private def parseXmlBoolean(s: String): Boolean = { + s.toLowerCase(Locale.ROOT) match { + case "true" | "1" => true + case "false" | "0" => false + case _ => throw new IllegalArgumentException(s"For input string: $s") + } + } + + private val supportedXmlDateFormatters = Seq( + // 2011-12-03 + // 2011-12-03+01:00 + DateTimeFormatter.ISO_DATE + ) + + private def parseXmlDate(value: String, options: XmlOptions): Date = { + val formatters = options.dateFormat.map(DateTimeFormatter.ofPattern). + map(supportedXmlDateFormatters :+ _).getOrElse(supportedXmlDateFormatters) + formatters.foreach { format => + try { + return Date.valueOf(LocalDate.parse(value, format)) + } catch { + case _: Exception => // continue + } + } + throw new IllegalArgumentException(s"cannot convert value $value to Date") + } + + private val supportedXmlTimestampFormatters = Seq( + // 2002-05-30 21:46:54 + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(DateTimeFormatter.ISO_LOCAL_TIME) + .toFormatter() + .withZone(ZoneId.of("UTC")), + // 2002-05-30T21:46:54 + DateTimeFormatter.ISO_LOCAL_DATE_TIME.withZone(ZoneId.of("UTC")), + // 2002-05-30T21:46:54+06:00 + DateTimeFormatter.ISO_OFFSET_DATE_TIME, + // 2002-05-30T21:46:54.1234Z + DateTimeFormatter.ISO_INSTANT + ) + + private def parseXmlTimestamp(value: String, options: XmlOptions): Timestamp = { + supportedXmlTimestampFormatters.foreach { format => + try { + return Timestamp.from(Instant.from(format.parse(value))) + } catch { + case _: Exception => // continue + } + } + options.timestampFormat.foreach { formatString => + // Check if there is offset or timezone and apply Spark timeZone if not + // Useful to support Java 8 and Java 11+ as they prioritize zone and offset differently + val hasTemporalInformation = formatString.indexOf("V") + + formatString.indexOf("z") + + formatString.indexOf("O") + + formatString.indexOf("X") + + formatString.indexOf("x") + + formatString.indexOf("Z") != (-6) + val format = if (hasTemporalInformation) { + DateTimeFormatter.ofPattern(formatString) + } else { + DateTimeFormatter.ofPattern(formatString).withZone(options.timezone.map(ZoneId.of).orNull) + } + try { + return Timestamp.from(Instant.from(format.parse(value))) + } catch { + case _: Exception => // continue + } + } + throw new IllegalArgumentException(s"cannot convert value $value to Timestamp") + } + + + // TODO: This function unnecessarily does type dispatch. Should merge it with `castTo`. + private[xml] def convertTo( + datum: String, + dataType: DataType, + options: XmlOptions): Any = { + val value = if (datum != null && options.ignoreSurroundingSpaces) { + datum.trim() + } else { + datum + } + if ((value == options.nullValue) || + (options.treatEmptyValuesAsNulls && value == "")) { + null + } else { + dataType match { + case NullType => castTo(value, StringType, options) + case LongType => signSafeToLong(value, options) + case DoubleType => signSafeToDouble(value, options) + case BooleanType => castTo(value, BooleanType, options) + case StringType => castTo(value, StringType, options) + case DateType => castTo(value, DateType, options) + case TimestampType => castTo(value, TimestampType, options) + case FloatType => signSafeToFloat(value, options) + case ByteType => castTo(value, ByteType, options) + case ShortType => castTo(value, ShortType, options) + case IntegerType => signSafeToInt(value, options) + case dt: DecimalType => castTo(value, dt, options) + case _ => throw new IllegalArgumentException( + s"Failed to parse a value for data type $dataType.") + } + } + } + + /** + * Helper method that checks and cast string representation of a numeric types. + */ + private[xml] def isBoolean(value: String): Boolean = { + value.toLowerCase(Locale.ROOT) match { + case "true" | "false" => true + case _ => false + } + } + + private[xml] def isDouble(value: String): Boolean = { + val signSafeValue = if (value.startsWith("+") || value.startsWith("-")) { + value.substring(1) + } else { + value + } + // Rule out strings ending in D or F, as they will parse as double but should be disallowed + if (value.nonEmpty && (value.last match { + case 'd' | 'D' | 'f' | 'F' => true + case _ => false + })) { + return false + } + (allCatch opt signSafeValue.toDouble).isDefined + } + + private[xml] def isInteger(value: String): Boolean = { + val signSafeValue = if (value.startsWith("+") || value.startsWith("-")) { + value.substring(1) + } else { + value + } + (allCatch opt signSafeValue.toInt).isDefined + } + + private[xml] def isLong(value: String): Boolean = { + val signSafeValue = if (value.startsWith("+") || value.startsWith("-")) { + value.substring(1) + } else { + value + } + (allCatch opt signSafeValue.toLong).isDefined + } + + private[xml] def isTimestamp(value: String, options: XmlOptions): Boolean = { + try { + parseXmlTimestamp(value, options) + true + } catch { + case _: IllegalArgumentException => false + } + } + + private[xml] def isDate(value: String, options: XmlOptions): Boolean = { + try { + parseXmlDate(value, options) + true + } catch { + case _: IllegalArgumentException => false + } + } + + private[xml] def signSafeToLong(value: String, options: XmlOptions): Long = { + if (value.startsWith("+")) { + val data = value.substring(1) + TypeCast.castTo(data, LongType, options).asInstanceOf[Long] + } else if (value.startsWith("-")) { + val data = value.substring(1) + -TypeCast.castTo(data, LongType, options).asInstanceOf[Long] + } else { + val data = value + TypeCast.castTo(data, LongType, options).asInstanceOf[Long] + } + } + + private[xml] def signSafeToDouble(value: String, options: XmlOptions): Double = { + if (value.startsWith("+")) { + val data = value.substring(1) + TypeCast.castTo(data, DoubleType, options).asInstanceOf[Double] + } else if (value.startsWith("-")) { + val data = value.substring(1) + -TypeCast.castTo(data, DoubleType, options).asInstanceOf[Double] + } else { + val data = value + TypeCast.castTo(data, DoubleType, options).asInstanceOf[Double] + } + } + + private[xml] def signSafeToInt(value: String, options: XmlOptions): Int = { + if (value.startsWith("+")) { + val data = value.substring(1) + TypeCast.castTo(data, IntegerType, options).asInstanceOf[Int] + } else if (value.startsWith("-")) { + val data = value.substring(1) + -TypeCast.castTo(data, IntegerType, options).asInstanceOf[Int] + } else { + val data = value + TypeCast.castTo(data, IntegerType, options).asInstanceOf[Int] + } + } + + private[xml] def signSafeToFloat(value: String, options: XmlOptions): Float = { + if (value.startsWith("+")) { + val data = value.substring(1) + TypeCast.castTo(data, FloatType, options).asInstanceOf[Float] + } else if (value.startsWith("-")) { + val data = value.substring(1) + -TypeCast.castTo(data, FloatType, options).asInstanceOf[Float] + } else { + val data = value + TypeCast.castTo(data, FloatType, options).asInstanceOf[Float] + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/ValidatorUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/ValidatorUtil.scala new file mode 100644 index 0000000000000..f3a5214ee0cda --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/ValidatorUtil.scala @@ -0,0 +1,55 @@ +/* + * 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.execution.datasources.xml.util + +import java.nio.file.Paths +import javax.xml.XMLConstants +import javax.xml.validation.{Schema, SchemaFactory} + +import com.google.common.cache.{CacheBuilder, CacheLoader} + +import org.apache.spark.SparkFiles + +/** + * Utilities for working with XSD validation. + */ +private[xml] object ValidatorUtil { + + // Parsing XSDs may be slow, so cache them by path: + + private val cache = CacheBuilder.newBuilder().softValues().build( + new CacheLoader[String, Schema] { + override def load(key: String): Schema = { + // Handle case where file exists as specified + var path = Paths.get(key) + if (!path.toFile.exists()) { + // Handle case where it was added with sc.addFile + path = Paths.get(SparkFiles.get(key)) + } + val schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI) + schemaFactory.newSchema(path.toFile) + } + }) + + /** + * Parses the XSD at the given local path and caches it. + * + * @param path path to XSD + * @return Schema for the file at that path + */ + def getSchema(path: String): Schema = cache.get(path) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchema.scala new file mode 100644 index 0000000000000..dc1c524b7a78d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchema.scala @@ -0,0 +1,280 @@ +/* + * 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.execution.datasources.xml.util + +import java.io.{File, FileInputStream, InputStreamReader, StringReader} +import java.nio.charset.StandardCharsets +import java.nio.file.Path + +import scala.collection.JavaConverters._ + +import org.apache.ws.commons.schema._ +import org.apache.ws.commons.schema.constants.Constants + +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.types._ + +/** + * Utility to generate a Spark schema from an XSD. Not all XSD schemas are simple tabular schemas, + * so not all elements or XSDs are supported. + */ +object XSDToSchema { + + /** + * Reads a schema from an XSD file. + * Note that if the schema consists of one complex parent type which you want to use as + * the row tag schema, then you will need to extract the schema of the single resulting + * struct in the resulting StructType, and use its StructType as your schema. + * + * @param xsdFile XSD file + * @return Spark-compatible schema + */ + def read(xsdFile: File): StructType = { + val xmlSchemaCollection = new XmlSchemaCollection() + xmlSchemaCollection.setBaseUri(xsdFile.getParent) + val xmlSchema = xmlSchemaCollection.read( + new InputStreamReader(new FileInputStream(xsdFile), StandardCharsets.UTF_8)) + + getStructType(xmlSchema) + } + + /** + * Reads a schema from an XSD file. + * Note that if the schema consists of one complex parent type which you want to use as + * the row tag schema, then you will need to extract the schema of the single resulting + * struct in the resulting StructType, and use its StructType as your schema. + * + * @param xsdFile XSD file + * @return Spark-compatible schema + */ + def read(xsdFile: Path): StructType = read(xsdFile.toFile) + + /** + * Reads a schema from an XSD as a string. + * Note that if the schema consists of one complex parent type which you want to use as + * the row tag schema, then you will need to extract the schema of the single resulting + * struct in the resulting StructType, and use its StructType as your schema. + * + * @param xsdString XSD as a string + * @return Spark-compatible schema + */ + def read(xsdString: String): StructType = { + val xmlSchema = new XmlSchemaCollection().read(new StringReader(xsdString)) + getStructType(xmlSchema) + } + + + private def getStructField(xmlSchema: XmlSchema, schemaType: XmlSchemaType): StructField = { + schemaType match { + // xs:simpleType + case simpleType: XmlSchemaSimpleType => + val schemaType = simpleType.getContent match { + case restriction: XmlSchemaSimpleTypeRestriction => + val qName = simpleType.getQName match { + case null => restriction.getBaseTypeName + case n => n + } + + // Hacky, is there a better way? see if the type is known as a custom + // type and use that if so, assuming along the way it's a simple restriction + val typeName = xmlSchema.getSchemaTypes.asScala.get(qName).map { s => + s.asInstanceOf[XmlSchemaSimpleType]. + getContent.asInstanceOf[XmlSchemaSimpleTypeRestriction].getBaseTypeName + }.getOrElse(qName) + + typeName match { + case Constants.XSD_BOOLEAN => BooleanType + case Constants.XSD_DECIMAL => + val facets = restriction.getFacets.asScala + val fracDigits = facets.collectFirst { + case facet: XmlSchemaFractionDigitsFacet => facet.getValue.toString.toInt + }.getOrElse(18) + val totalDigits = facets.collectFirst { + case facet: XmlSchemaTotalDigitsFacet => facet.getValue.toString.toInt + }.getOrElse(38) + DecimalType(totalDigits, math.min(totalDigits, fracDigits)) + case Constants.XSD_UNSIGNEDLONG => DecimalType(38, 0) + case Constants.XSD_DOUBLE => DoubleType + case Constants.XSD_FLOAT => FloatType + case Constants.XSD_BYTE => ByteType + case Constants.XSD_SHORT | + Constants.XSD_UNSIGNEDBYTE => ShortType + case Constants.XSD_INTEGER | + Constants.XSD_NEGATIVEINTEGER | + Constants.XSD_NONNEGATIVEINTEGER | + Constants.XSD_NONPOSITIVEINTEGER | + Constants.XSD_POSITIVEINTEGER | + Constants.XSD_UNSIGNEDSHORT => IntegerType + case Constants.XSD_LONG | + Constants.XSD_UNSIGNEDINT => LongType + case Constants.XSD_DATE => DateType + case Constants.XSD_DATETIME => TimestampType + case _ => StringType + } + case _ => StringType + } + StructField("baseName", schemaType) + + // xs:complexType + case complexType: XmlSchemaComplexType => + complexType.getContentModel match { + case content: XmlSchemaSimpleContent => + // xs:simpleContent + content.getContent match { + case extension: XmlSchemaSimpleContentExtension => + val baseStructField = getStructField(xmlSchema, + xmlSchema.getParent.getTypeByQName(extension.getBaseTypeName)) + val value = StructField("_VALUE", baseStructField.dataType) + val attributes = extension.getAttributes.asScala.map { + case attribute: XmlSchemaAttribute => + val baseStructField = getStructField(xmlSchema, + xmlSchema.getParent.getTypeByQName(attribute.getSchemaTypeName)) + StructField(s"_${attribute.getName}", baseStructField.dataType, + attribute.getUse != XmlSchemaUse.REQUIRED) + }.toSeq + StructField(complexType.getName, StructType(value +: attributes)) + case unsupported => + throw new IllegalArgumentException(s"Unsupported content: $unsupported") + } + case content: XmlSchemaComplexContent => + val complexContent = content.getContent + complexContent match { + case extension: XmlSchemaComplexContentExtension => + val baseStructField = getStructField(xmlSchema, + xmlSchema.getParent.getTypeByQName(extension.getBaseTypeName)) + val baseFields = baseStructField.dataType match { + case structType: StructType => structType.fields + case others => + throw new IllegalArgumentException( + s"Non-StructType in ComplexContentExtension: $others" + ) + } + + val extendedFields = getStructFieldsFromParticle(extension.getParticle, xmlSchema) + StructField( + schemaType.getQName.getLocalPart, + StructType(baseFields ++ extendedFields) + ) + case unsupported => + throw new IllegalArgumentException(s"Unsupported content: $unsupported") + } + case null => + val childFields = getStructFieldsFromParticle(complexType.getParticle, xmlSchema) + val attributes = complexType.getAttributes.asScala.map { + case attribute: XmlSchemaAttribute => + val attributeType = attribute.getSchemaTypeName match { + case null => + StringType + case t => + getStructField(xmlSchema, xmlSchema.getParent.getTypeByQName(t)).dataType + } + StructField(s"_${attribute.getName}", attributeType, + attribute.getUse != XmlSchemaUse.REQUIRED) + }.toSeq + StructField(complexType.getName, StructType(childFields ++ attributes)) + case unsupported => + throw new IllegalArgumentException(s"Unsupported content model: $unsupported") + } + case unsupported => + throw new IllegalArgumentException(s"Unsupported schema element type: $unsupported") + } + } + + private def getStructType(xmlSchema: XmlSchema): StructType = { + StructType(xmlSchema.getElements.asScala.toSeq.map { case (_, schemaElement) => + val schemaType = schemaElement.getSchemaType + // if (schemaType.isAnonymous) { + // schemaType.setName(qName.getLocalPart) + // } + val rootType = getStructField(xmlSchema, schemaType) + StructField(schemaElement.getName, rootType.dataType, schemaElement.getMinOccurs == 0) + }) + } + + private def getStructFieldsFromParticle( + particle: XmlSchemaParticle, + xmlSchema: XmlSchema + ): Seq[StructField] = { + particle match { + // xs:all + case all: XmlSchemaAll => + all.getItems.asScala.map { + case element: XmlSchemaElement => + val baseStructField = getStructField(xmlSchema, element.getSchemaType) + val nullable = element.getMinOccurs == 0 + if (element.getMaxOccurs == 1) { + StructField(element.getName, baseStructField.dataType, nullable) + } else { + StructField(element.getName, ArrayType(baseStructField.dataType), nullable) + } + }.toSeq + // xs:choice + case choice: XmlSchemaChoice => + choice.getItems.asScala.map { + case element: XmlSchemaElement => + val baseStructField = getStructField(xmlSchema, element.getSchemaType) + if (element.getMaxOccurs == 1) { + StructField(element.getName, baseStructField.dataType, true) + } else { + StructField(element.getName, ArrayType(baseStructField.dataType), true) + } + case any: XmlSchemaAny => + val dataType = if (any.getMaxOccurs > 1) ArrayType(StringType) else StringType + StructField(XmlOptions.DEFAULT_WILDCARD_COL_NAME, dataType, true) + }.toSeq + // xs:sequence + case sequence: XmlSchemaSequence => + // flatten xs:choice nodes + sequence.getItems.asScala.flatMap { + _ match { + case choice: XmlSchemaChoice => + choice.getItems.asScala.map { e => + val xme = e.asInstanceOf[XmlSchemaElement] + val baseType = getStructField(xmlSchema, xme.getSchemaType).dataType + val dataType = if (xme.getMaxOccurs > 1) ArrayType(baseType) else baseType + StructField(xme.getName, dataType, true) + } + case e: XmlSchemaElement => + val refQName = e.getRef.getTargetQName + val baseType = + if (refQName != null) { + getStructField( + xmlSchema, + xmlSchema.getParent.getElementByQName(refQName).getSchemaType).dataType + } + else getStructField(xmlSchema, e.getSchemaType).dataType + val dataType = if (e.getMaxOccurs > 1) ArrayType(baseType) else baseType + val nullable = e.getMinOccurs == 0 + val structFieldName = + Option(refQName).map(_.getLocalPart).getOrElse(e.getName) + Seq(StructField(structFieldName, dataType, nullable)) + case any: XmlSchemaAny => + val dataType = + if (any.getMaxOccurs > 1) ArrayType(StringType) else StringType + val nullable = any.getMinOccurs == 0 + Seq(StructField(XmlOptions.DEFAULT_WILDCARD_COL_NAME, dataType, nullable)) + case unsupported => + throw new IllegalArgumentException(s"Unsupported item: $unsupported") + } + }.toSeq + case null => + Seq.empty + case unsupported => + throw new IllegalArgumentException(s"Unsupported particle: $unsupported") + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFile.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFile.scala new file mode 100644 index 0000000000000..ef73652d9de6d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFile.scala @@ -0,0 +1,163 @@ +/* + * 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.execution.datasources.xml.util + +import java.io.CharArrayWriter +import java.nio.charset.Charset +import javax.xml.stream.XMLOutputFactory + +import scala.collection.Map + +import com.sun.xml.txw2.output.IndentingXMLStreamWriter +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.io.compress.CompressionCodec + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.util.CompressionCodecs +import org.apache.spark.sql.execution.datasources.xml.{XmlInputFormat, XmlOptions} +import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlGenerator + +private[xml] object XmlFile { + val DEFAULT_INDENT = " " + + def withCharset( + context: SparkContext, + location: String, + charset: String, + rowTag: String): RDD[String] = { + // This just checks the charset's validity early, to keep behavior + Charset.forName(charset) + val config = new Configuration(context.hadoopConfiguration) + config.set(XmlInputFormat.START_TAG_KEY, s"<$rowTag>") + config.set(XmlInputFormat.END_TAG_KEY, s"") + config.set(XmlInputFormat.ENCODING_KEY, charset) + context.newAPIHadoopFile(location, + classOf[XmlInputFormat], + classOf[LongWritable], + classOf[Text], + config).map { case (_, text) => text.toString } + } + + /** + * Note that writing a XML file from [[DataFrame]] having a field + * [[org.apache.spark.sql.types.ArrayType]] with its element as nested array would have + * an additional nested field for the element. For example, the [[DataFrame]] having + * a field below, + * + * fieldA Array(Array(data1, data2)) + * + * would produce a XML file below. + * + * + * data1 + * + * + * data2 + * + * + * Namely, roundtrip in writing and reading can end up in different schema structure. + */ + def saveAsXmlFile( + dataFrame: DataFrame, + path: String, + parameters: Map[String, String] = Map()): Unit = { + val options = XmlOptions(parameters.toMap) + val codec = Option(options.codec).map(CompressionCodecs.getCodecClassName) + // scalastyle:off classforname + val codecClass: Option[Class[_ <: CompressionCodec]] = + codec.map(Class.forName(_).asInstanceOf[Class[CompressionCodec]]) + val rowSchema = dataFrame.schema + val indent = XmlFile.DEFAULT_INDENT + + // Allow a root tag to be like "rootTag foo='bar'" + // This is hacky; won't deal correctly with spaces in attributes, but want + // to make this at least work for simple cases without much complication + val rootTagTokens = options.rootTag.split(" ") + val rootElementName = rootTagTokens.head + val rootAttributes: Map[String, String] = + if (rootTagTokens.length > 1) { + rootTagTokens.tail.map { kv => + val Array(k, v) = kv.split("=") + k -> v.replaceAll("['\"]", "") + }.toMap + } else { + Map.empty + } + val declaration = options.declaration + + val xmlRDD = dataFrame.rdd.mapPartitions { iter => + val factory = XMLOutputFactory.newInstance() + val writer = new CharArrayWriter() + val xmlWriter = factory.createXMLStreamWriter(writer) + val indentingXmlWriter = new IndentingXMLStreamWriter(xmlWriter) + indentingXmlWriter.setIndentStep(indent) + + new Iterator[String] { + var firstRow: Boolean = true + var lastRow: Boolean = true + + override def hasNext: Boolean = iter.hasNext || firstRow || lastRow + + override def next(): String = { + if (iter.nonEmpty) { + if (firstRow) { + if (declaration != null && declaration.nonEmpty) { + indentingXmlWriter.writeProcessingInstruction("xml", declaration) + indentingXmlWriter.writeCharacters("\n") + } + indentingXmlWriter.writeStartElement(rootElementName) + rootAttributes.foreach { case (k, v) => + indentingXmlWriter.writeAttribute(k, v) + } + firstRow = false + } + val xml = { + StaxXmlGenerator( + rowSchema, + indentingXmlWriter, + options)(iter.next()) + indentingXmlWriter.flush() + writer.toString + } + writer.reset() + xml + } else { + if (!firstRow) { + lastRow = false + indentingXmlWriter.writeEndElement() + indentingXmlWriter.close() + writer.toString + } else { + // This means the iterator was initially empty. + firstRow = false + lastRow = false + "" + } + } + } + } + } + + codecClass match { + case None => xmlRDD.saveAsTextFile(path) + case Some(codec) => xmlRDD.saveAsTextFile(path, codec) + } + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/xml/JavaXmlSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/xml/JavaXmlSuite.java new file mode 100644 index 0000000000000..dae6d93a37b62 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/xml/JavaXmlSuite.java @@ -0,0 +1,111 @@ +/* + * 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 test.org.apache.spark.sql.execution.datasources.xml; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.datasources.xml.XmlOptions; +import org.apache.spark.sql.execution.datasources.xml.XmlReader; + +public final class JavaXmlSuite { + + private static final int numBooks = 12; + private static final String booksFile = "src/test/resources/test-data/xml-resources/books.xml"; + private static final String booksFileTag = "book"; + + private SparkSession spark; + private Path tempDir; + + private static void setEnv(String key, String value) { + try { + Map env = System.getenv(); + Class cl = env.getClass(); + Field field = cl.getDeclaredField("m"); + field.setAccessible(true); + Map writableEnv = (Map) field.get(env); + writableEnv.put(key, value); + } catch (Exception e) { + throw new IllegalStateException("Failed to set environment variable", e); + } + } + + @Before + public void setUp() throws IOException { + setEnv("SPARK_LOCAL_IP", "127.0.0.1"); + spark = SparkSession.builder() + .master("local[2]") + .appName("XmlSuite") + .config("spark.ui.enabled", false) + .getOrCreate(); + spark.sparkContext().setLogLevel("WARN"); + tempDir = Files.createTempDirectory("JavaXmlSuite"); + tempDir.toFile().deleteOnExit(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + private Path getEmptyTempDir() throws IOException { + return Files.createTempDirectory(tempDir, "test"); + } + + @Test + public void testXmlParser() { + Dataset df = (new XmlReader()).withRowTag(booksFileTag).xmlFile(spark, booksFile); + String prefix = XmlOptions.DEFAULT_ATTRIBUTE_PREFIX(); + long result = df.select(prefix + "id").count(); + Assert.assertEquals(result, numBooks); + } + + @Test + public void testLoad() { + Map options = new HashMap<>(); + options.put("rowTag", booksFileTag); + Dataset df = spark.read().options(options).format("xml").load(booksFile); + long result = df.select("description").count(); + Assert.assertEquals(result, numBooks); + } + + @Test + public void testSave() throws IOException { + Path booksPath = getEmptyTempDir().resolve("booksFile"); + + Dataset df = (new XmlReader()).withRowTag(booksFileTag).xmlFile(spark, booksFile); + df.select("price", "description").write().format("xml").save(booksPath.toString()); + + Dataset newDf = (new XmlReader()).xmlFile(spark, booksPath.toString()); + long result = newDf.select("price").count(); + Assert.assertEquals(result, numBooks); + } + +} diff --git a/sql/core/src/test/resources/test-data/xml-resources/ages-mixed-types.xml b/sql/core/src/test/resources/test-data/xml-resources/ages-mixed-types.xml new file mode 100644 index 0000000000000..e0a696221e890 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/ages-mixed-types.xml @@ -0,0 +1,15 @@ + + + 25 + Hyukjin + + + 30 + Lars + + + 30 + Cheng + Lian + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/ages-with-spaces.xml b/sql/core/src/test/resources/test-data/xml-resources/ages-with-spaces.xml new file mode 100644 index 0000000000000..2ad849b5078bb --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/ages-with-spaces.xml @@ -0,0 +1,20 @@ + + + 25 + + + Hyukjin + + + + + 30 + Lars + + + 30 + Lion + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/ages.xml b/sql/core/src/test/resources/test-data/xml-resources/ages.xml new file mode 100644 index 0000000000000..29b23fc648650 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/ages.xml @@ -0,0 +1,14 @@ + + + 25 + Hyukjin + + + 30 + Lars + + + 30 + Lion + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLine.xml b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLine.xml new file mode 100644 index 0000000000000..cdc29f75d972d --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLine.xml @@ -0,0 +1,11 @@ + + + + Tove + Jani + Reminder + Don't forget me this weekend! + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineCR.xml b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineCR.xml new file mode 100644 index 0000000000000..ff09aadb65db6 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineCR.xml @@ -0,0 +1 @@ + Tove Jani Reminder Don't forget me this weekend! \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineLF.xml b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineLF.xml new file mode 100644 index 0000000000000..cdc29f75d972d --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/attributesStartWithNewLineLF.xml @@ -0,0 +1,11 @@ + + + + Tove + Jani + Reminder + Don't forget me this weekend! + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/basket.xml b/sql/core/src/test/resources/test-data/xml-resources/basket.xml new file mode 100644 index 0000000000000..9a14c212212dd --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/basket.xml @@ -0,0 +1,12 @@ + + + + 9027 + glasstop stove in black + + + 288 + wooden spoon + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/basket.xsd b/sql/core/src/test/resources/test-data/xml-resources/basket.xsd new file mode 100644 index 0000000000000..7b19a24286715 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/basket.xsd @@ -0,0 +1,17 @@ + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/basket_invalid.xml b/sql/core/src/test/resources/test-data/xml-resources/basket_invalid.xml new file mode 100644 index 0000000000000..cdb1b15969840 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/basket_invalid.xml @@ -0,0 +1,14 @@ + + + + 9027 + glasstop stove in black + 123 + + + 288 + wooden spoon + 123 + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-attributes-in-no-child.xml b/sql/core/src/test/resources/test-data/xml-resources/books-attributes-in-no-child.xml new file mode 100644 index 0000000000000..68b8605efa7f1 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-attributes-in-no-child.xml @@ -0,0 +1,75 @@ + + + + Gambardella, Matthew + XML Developer's Guide + twenty + 2000-10-01 + + + Ralls, Kim + Midnight Rain + 5.95 + 2000-12-16 + + + Corets, Eva + Maeve Ascendant + 5.95 + 2000-11-17 + + + Corets, Eva + Oberon's Legacy + 5.95 + 2001-03-10 + + + Corets, Eva + The Sundered Grail + 5.95 + 2001-09-10 + + + Randall, Cynthia + Lover Birds + 4.95 + 2000-09-02 + + + Thurman, Paula + Splish Splash + 4.95 + 2000-11-02 + + + Knorr, Stefan + Creepy Crawlies + 4.95 + 2000-12-06 + + + Kress, Peter + Paradox Lost + 6.95 + 2000-11-02 + + + O'Brien, Tim + Microsoft .NET: The Programming Bible + 36.95 + 2000-12-09 + + + O'Brien, Tim + MSXML3: A Comprehensive Guide + 36.95 + 2000-12-01 + + + Galos, Mike + Visual Studio 7: A Comprehensive Guide + 49.95 + 2001-04-16 + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-complicated-null-attribute.xml b/sql/core/src/test/resources/test-data/xml-resources/books-complicated-null-attribute.xml new file mode 100644 index 0000000000000..f256cd61ba46b --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-complicated-null-attribute.xml @@ -0,0 +1,60 @@ + + + + Gambardella, Matthew + XML Developer's Guide + + 1 + Computer + + 44.95 + + + 2000 + 10 + 01 + + + + + Ralls, Kim + Midnight Rain + + 2 + Fantasy + + 5.95 + + + 2000 + 12 + 16 + + + 2000 + 12 + 17 + + + + + Corets, Eva + Maeve Ascendant + + 2 + Fantasy + + + + 2000 + 11 + 07 + + + 2000 + 11 + 08 + + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-complicated.xml b/sql/core/src/test/resources/test-data/xml-resources/books-complicated.xml new file mode 100644 index 0000000000000..d21d86305fcdf --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-complicated.xml @@ -0,0 +1,60 @@ + + + + Gambardella, Matthew + XML Developer's Guide + + 1 + Computer + + 44.95 + + + 2000 + 10 + 01 + + + + + Ralls, Kim + Midnight Rain + + 2 + Fantasy + + 5.95 + + + 2000 + 12 + 16 + + + 2000 + 12 + 17 + + + + + Corets, Eva + Maeve Ascendant + + 2 + Fantasy + + + + 2000 + 11 + 07 + + + 2000 + 11 + 08 + + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-malformed-attributes.xml b/sql/core/src/test/resources/test-data/xml-resources/books-malformed-attributes.xml new file mode 100644 index 0000000000000..e9830d55d3da7 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-malformed-attributes.xml @@ -0,0 +1,43 @@ + + + + Kress, Peter + Paradox Lost + Science Fiction + 6.95 + 2000-11-02 + After an inadvertant trip through a Heisenberg + Uncertainty Device, James Salway discovers the problems + of being quantum. + + + O'Brien, Tim + Microsoft .NET: The Programming Bible + Computer + 36.95 + 2000-12-09 + Microsoft's .NET initiative is explored in + detail in this deep programmer's reference. + + + O'Brien, Tim + MSXML3: A Comprehensive Guide + Computer + 36.95 + 2000-12-01 + The Microsoft MSXML3 parser is covered in + detail, with attention to XML DOM interfaces, XSLT processing, + SAX and more. + + + Galos, Mike + Visual Studio 7: A Comprehensive Guide + Computer + 49.95 + 2001-04-16 + Microsoft Visual Studio 7 is explored in depth, + looking at how Visual Basic, Visual C++, C#, and ASP+ are + integrated into a comprehensive development + environment. + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-namespaces.xml b/sql/core/src/test/resources/test-data/xml-resources/books-namespaces.xml new file mode 100644 index 0000000000000..34ef4f8728a40 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-namespaces.xml @@ -0,0 +1,12 @@ + + + + Gambardella, Matthew + + + Ralls, Kim + + + Corets, Eva + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-nested-array.xml b/sql/core/src/test/resources/test-data/xml-resources/books-nested-array.xml new file mode 100644 index 0000000000000..7a52bdb75b028 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-nested-array.xml @@ -0,0 +1,130 @@ + + + + Gambardella, Matthew + XML Developer's Guide + Computer + 44.952000-10-01 + 2000-10-02 + An in-depth look at creating applications + with XML. + + + Ralls, Kim + Midnight RainFantasy + 5.95 + 2000-12-16 + 2000-12-17 + A former architect battles corporate zombies, + an evil sorceress, and her own childhood to become queen + of the world. + + + Corets, Eva + Maeve Ascendant + Fantasy + 5.95 + 2000-11-17 + 2000-11-18 + After the collapse of a nanotechnology + society in England, the young survivors lay the + foundation for a new society. + + + Corets, Eva + Oberon's Legacy + Fantasy + 5.95 + 2001-03-10 + 2001-03-11 + In post-apocalypse England, the mysterious + agent known only as Oberon helps to create a new life + for the inhabitants of London. Sequel to Maeve + Ascendant. + + + Corets, Eva + The Sundered Grail + Fantasy + 5.95 + 2001-09-10 + 2001-09-11 + The two daughters of Maeve, half-sisters, + battle one another for control of England. Sequel to + Oberon's Legacy. + + + Randall, Cynthia + Lover Birds + Romance + 4.95 + 2000-09-02 + 2000-09-03 + When Carla meets Paul at an ornithology + conference, tempers fly as feathers get ruffled. + + + Thurman, Paula + Splish Splash + Romance + 4.95 + 2000-11-02 + 2000-11-03 + A deep sea diver finds true love twenty + thousand leagues beneath the sea. + + + Knorr, Stefan + Creepy Crawlies + Horror + 4.95 + 2000-12-06 + 2000-12-07 + An anthology of horror stories about roaches, + centipedes, scorpions and other insects. + + + Kress, Peter + Paradox Lost + Science Fiction + 6.95 + 2000-11-02 + 2000-11-03 + After an inadvertant trip through a Heisenberg + Uncertainty Device, James Salway discovers the problems + of being quantum. + + + O'Brien, Tim + Microsoft .NET: The Programming Bible + Computer + 36.95 + 2000-12-09 + 2000-12-10 + Microsoft's .NET initiative is explored in + detail in this deep programmer's reference. + + + O'Brien, Tim + MSXML3: A Comprehensive Guide + Computer + 36.95 + 2000-12-01 + 2000-12-02 + The Microsoft MSXML3 parser is covered in + detail, with attention to XML DOM interfaces, XSLT processing, + SAX and more. + + + Galos, Mike + Visual Studio 7: A Comprehensive Guide + Computer + 49.95 + 2001-04-16 + 2001-04-17 + Microsoft Visual Studio 7 is explored in depth, + looking at how Visual Basic, Visual C++, C#, and ASP+ are + integrated into a comprehensive development + environment. + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-nested-object.xml b/sql/core/src/test/resources/test-data/xml-resources/books-nested-object.xml new file mode 100644 index 0000000000000..5754aa06325eb --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-nested-object.xml @@ -0,0 +1,144 @@ + + + + Gambardella, Matthew + XML Developer's Guide + Computer + 44.95 + + 2000-10-01 + + An in-depth look at creating applications + with XML. + + + Ralls, Kim + Midnight Rain + Fantasy + 5.95 + + 2000-12-16 + + A former architect battles corporate zombies, + an evil sorceress, and her own childhood to become queen + of the world. + + + Corets, Eva + Maeve Ascendant + Fantasy + 5.95 + + 2000-11-17 + + After the collapse of a nanotechnology + society in England, the young survivors lay the + foundation for a new society. + + + Corets, Eva + Oberon's Legacy + Fantasy + 5.95 + + 2001-03-10 + + In post-apocalypse England, the mysterious + agent known only as Oberon helps to create a new life + for the inhabitants of London. Sequel to Maeve + Ascendant. + + + Corets, Eva + The Sundered Grail + Fantasy + 5.95 + + 2001-09-10 + + The two daughters of Maeve, half-sisters, + battle one another for control of England. Sequel to + Oberon's Legacy. + + + Randall, Cynthia + Lover Birds + Romance + 4.95 + + 2000-09-02 + + When Carla meets Paul at an ornithology + conference, tempers fly as feathers get ruffled. + + + Thurman, Paula + Splish Splash + Romance + 4.95 + + 2000-11-02 + + A deep sea diver finds true love twenty + thousand leagues beneath the sea. + + + Knorr, Stefan + Creepy Crawlies + Horror + 4.95 + + 2000-12-06 + + An anthology of horror stories about roaches, + centipedes, scorpions and other insects. + + + Kress, Peter + Paradox Lost + Science Fiction + 6.95 + + 2000-11-02 + + After an inadvertant trip through a Heisenberg + Uncertainty Device, James Salway discovers the problems + of being quantum. + + + O'Brien, Tim + Microsoft .NET: The Programming Bible + Computer + 36.95 + + 2000-11-02 + + Microsoft's .NET initiative is explored in + detail in this deep programmer's reference. + + + O'Brien, Tim + MSXML3: A Comprehensive Guide + Computer + 36.95 + + 2000-12-01 + + The Microsoft MSXML3 parser is covered in + detail, with attention to XML DOM interfaces, XSLT processing, + SAX and more. + + + Galos, Mike + Visual Studio 7: A Comprehensive Guide + Computer + 49.95 + + 2001-04-16 + + Microsoft Visual Studio 7 is explored in depth, + looking at how Visual Basic, Visual C++, C#, and ASP+ are + integrated into a comprehensive development + environment. + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books-unicode-in-tag-name.xml b/sql/core/src/test/resources/test-data/xml-resources/books-unicode-in-tag-name.xml new file mode 100644 index 0000000000000..13d325cc3f1a8 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books-unicode-in-tag-name.xml @@ -0,0 +1,24 @@ + +<κατάλογος> + <書 ид="bk101"> + Gambardella, Matthew + <ítulo>XML Developer's Guide + <ჟანრი>Computer + <цена>44.95 + 2000-10-01 + + <書> + Ralls, Kim + <ítulo>Midnight Rain + <ჟანრი>Fantasy + <цена>5.95 + 2000-12-16 + + <書 ид="bk103"> + Corets, Eva + <ítulo>Maeve Ascendant + <ჟანრი>Fantasy + <цена>5.95 + 2000-11-17 + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/books.xml b/sql/core/src/test/resources/test-data/xml-resources/books.xml new file mode 100644 index 0000000000000..db13defd22ced --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/books.xml @@ -0,0 +1,136 @@ + + + + Gambardella, Matthew + XML Developer's Guide + Computer + 44.95 + 2000-10-01 + + + + An in-depth look at creating applications + with XML.This manual describes Oracle XML DB, and how you can use it to store, generate, manipulate, manage, + and query XML data in the database. + + + After introducing you to the heart of Oracle XML DB, namely the XMLType framework and Oracle XML DB repository, + the manual provides a brief introduction to design criteria to consider when planning your Oracle XML DB + application. It provides examples of how and where you can use Oracle XML DB. + + + The manual then describes ways you can store and retrieve XML data using Oracle XML DB, APIs for manipulating + XMLType data, and ways you can view, generate, transform, and search on existing XML data. The remainder of + the manual discusses how to use Oracle XML DB repository, including versioning and security, + how to access and manipulate repository resources using protocols, SQL, PL/SQL, or Java, and how to manage + your Oracle XML DB application using Oracle Enterprise Manager. It also introduces you to XML messaging and + Oracle Streams Advanced Queuing XMLType support. + + Ralls, Kim + Midnight Rain + Fantasy + 5.95 + 2000-12-16 + A former architect battles corporate zombies, + an evil sorceress, and her own childhood to become queen + of the world. + + + Corets, Eva + Maeve Ascendant + Fantasy + 5.95 + 2000-11-17 + After the collapse of a nanotechnology + society in England, the young survivors lay the + foundation for a new society. + + + Corets, Eva + Oberon's Legacy + Fantasy + 5.95 + 2001-03-10 + In post-apocalypse England, the mysterious + agent known only as Oberon helps to create a new life + for the inhabitants of London. Sequel to Maeve + Ascendant. + + + Corets, Eva + The Sundered Grail + Fantasy + 5.95 + 2001-09-10 + The two daughters of Maeve, half-sisters, + battle one another for control of England. Sequel to + Oberon's Legacy. + + + Randall, Cynthia + Lover Birds + Romance + 4.95 + 2000-09-02 + When Carla meets Paul at an ornithology + conference, tempers fly as feathers get ruffled. + + + Thurman, Paula + Splish Splash + Romance + 4.95 + 2000-11-02 + A deep sea diver finds true love twenty + thousand leagues beneath the sea. + + + Knorr, Stefan + Creepy Crawlies + Horror + 4.95 + 2000-12-06 + An anthology of horror stories about roaches, + centipedes, scorpions and other insects. + + + Kress, Peter + Paradox Lost + Science Fiction + 6.95 + 2000-11-02 + After an inadvertant trip through a Heisenberg + Uncertainty Device, James Salway discovers the problems + of being quantum. + + + O'Brien, Tim + Microsoft .NET: The Programming Bible + Computer + 36.95 + 2000-12-09 + Microsoft's .NET initiative is explored in + detail in this deep programmer's reference. + + + O'Brien, Tim + MSXML3: A Comprehensive Guide + Computer + 36.95 + 2000-12-01 + The Microsoft MSXML3 parser is covered in + detail, with attention to XML DOM interfaces, XSLT processing, + SAX and more. + + + Galos, Mike + Visual Studio 7: A Comprehensive Guide + Computer + 49.95 + 2001-04-16 + Microsoft Visual Studio 7 is explored in depth, + looking at how Visual Basic, Visual C++, C#, and ASP+ are + integrated into a comprehensive development + environment. + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-attribute.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-attribute.xml new file mode 100644 index 0000000000000..4cd772c910f61 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-attribute.xml @@ -0,0 +1,9 @@ + + + + 2015 + Chevy + Volt + No + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-iso-8859-1.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-iso-8859-1.xml new file mode 100644 index 0000000000000..649e880e87b07 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-iso-8859-1.xml @@ -0,0 +1,21 @@ + + + + 2012 + Tesla + S + No comment + + + 1997 + Ford + E350 + Go get one now they are going fast + + + 2015 + Chevy + Volt + No + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-malformed.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-malformed.xml new file mode 100644 index 0000000000000..3859f04fbe199 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-malformed.xml @@ -0,0 +1,20 @@ + + + + 2012 + Tesla + >S + No comment + + + + Ford + E350model> + Go get one now they are going fast + + + 2015 + Chevy + Volt + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-mixed-attr-no-child.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-mixed-attr-no-child.xml new file mode 100644 index 0000000000000..91bd46de2cc75 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-mixed-attr-no-child.xml @@ -0,0 +1,25 @@ + + + + 2012-12-01 + Tesla + S + No comment + + + + 2012> + 11 + 2 + + Ford + E350 + Go get one now they are going fast + + + 2015 + Chevy + Volt + No + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-no-indentation.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-no-indentation.xml new file mode 100644 index 0000000000000..d603759da38bb --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-no-indentation.xml @@ -0,0 +1,2 @@ + +2012TeslaSNo comment1997FordE350Go get one now they are going fast2015ChevyVoltNo diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars-unbalanced-elements.xml b/sql/core/src/test/resources/test-data/xml-resources/cars-unbalanced-elements.xml new file mode 100644 index 0000000000000..68324f58f4a05 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars-unbalanced-elements.xml @@ -0,0 +1,19 @@ + + + + green + Tesla + S + No orgment + 2015 + + + Ford + E350 + Go get one now they are going fast + + + Volt + Go get one now they are going fast + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars.xml b/sql/core/src/test/resources/test-data/xml-resources/cars.xml new file mode 100644 index 0000000000000..ea17d5e2174b8 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/cars.xml @@ -0,0 +1,21 @@ + + + + 2012 + Tesla + S + No comment + + + 1997 + Ford + E350 + Go get one now they are going fast + + + 2015 + Chevy + Volt + No + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/cars.xml.bz2 b/sql/core/src/test/resources/test-data/xml-resources/cars.xml.bz2 new file mode 100644 index 0000000000000000000000000000000000000000..bbe95a96cabd75e62797bb243494d18c365673b9 GIT binary patch literal 229 zcmV7T4*^jL0KkKS!D}ra{vHVUw}jqPyza38q~4$=m;gZ20Wqi~lpxbh7}PY$pfqS`^)f;rnFwJaL?Fig?P&ZjrfKN_oyON? zCt0i9l;mjyZI68(^K?7C7vT=goI*}%3he=@p za1p#x3ftN>p=;tMbH!k}3PiA`=aR<9s-?owNZ~^j3Wbp@Vo70PM8}X0I5!AI%K$`V faS(jpsMKrkVu&FvyVXpBA^5wJDZ+$QX{04s}wG>`ctvnl{c=q&ai1G9FJ7gzt%PGU_NuX M2QuO`%h>?{0E)3;1^@s6 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/xml-resources/catalog.xsd b/sql/core/src/test/resources/test-data/xml-resources/catalog.xsd new file mode 100644 index 0000000000000..4af4037df18a3 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/catalog.xsd @@ -0,0 +1,41 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/choice.xsd b/sql/core/src/test/resources/test-data/xml-resources/choice.xsd new file mode 100644 index 0000000000000..6c7269966aad5 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/choice.xsd @@ -0,0 +1,12 @@ + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/complex-content-extension.xsd b/sql/core/src/test/resources/test-data/xml-resources/complex-content-extension.xsd new file mode 100644 index 0000000000000..f13719304fbb7 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/complex-content-extension.xsd @@ -0,0 +1,25 @@ + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/datatypes-valid-and-invalid.xml b/sql/core/src/test/resources/test-data/xml-resources/datatypes-valid-and-invalid.xml new file mode 100644 index 0000000000000..b205eba081190 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/datatypes-valid-and-invalid.xml @@ -0,0 +1,31 @@ + + + + 10 + 10 + 10.0 + 10.0 + true + Ten + 1 + 2 + + 345 + 123 + + + + Ten + Ten + Ten + Ten + Ten + Ten + Ten + 2 + + 345 + OneTwoThree + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/date.xml b/sql/core/src/test/resources/test-data/xml-resources/date.xml new file mode 100644 index 0000000000000..8a5e5cdb92652 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/date.xml @@ -0,0 +1,5 @@ + + John Smith + 2021-02-01 + 02-01-2021 + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/decimal-with-restriction.xsd b/sql/core/src/test/resources/test-data/xml-resources/decimal-with-restriction.xsd new file mode 100644 index 0000000000000..e60cc548e2894 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/decimal-with-restriction.xsd @@ -0,0 +1,18 @@ + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/empty.xml b/sql/core/src/test/resources/test-data/xml-resources/empty.xml new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/core/src/test/resources/test-data/xml-resources/feed-with-spaces.xml b/sql/core/src/test/resources/test-data/xml-resources/feed-with-spaces.xml new file mode 100644 index 0000000000000..bc6e227a1325e --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/feed-with-spaces.xml @@ -0,0 +1,15 @@ + + + + A + + + B + + + C + + + D + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml b/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml new file mode 100644 index 0000000000000..7164f067a7d28 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml @@ -0,0 +1,3621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.bz2 b/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.bz2 new file mode 100644 index 0000000000000000000000000000000000000000..0003edd5dc293604c9d045f5c41b55c5643775a4 GIT binary patch literal 30761 zcmaI83s{K--x$hqP*q91@izCQaIDhqUR7I*m~|6WoFaj4__- zG3jK5&X~8IwSo}@PYn%P6i)~qP_(k-+X2KxR48oCRCX{yYo-`AYEAOpq%+^z*WTB) zO)eFL_xL=|ec%7z|K?q0*8cs!XGA139D>cckpFt`Z&>Mn{_$U)JRJS|-+udcYx6tr zJpXk_NXWMbN49_aa@z-?A)#-+5%K~&6Y}ikkllZTpI?M1-~Hsrr{KYxyF)@kpLr+b z4}Ne-iSn zrR7h5G_1dfI>Q87ti8JwEeJgRztK@&s^Sh)LWyya^qa;`HAN;97 z+4Vyx;Sc{~{+Rpw=3PJF3hy;KBX9M*m9#3SksCZ8H`9EXM-uM--_3tLvyW+Ctce<4 zIoKS3b=-RPpgb|MdcrP>Vc(=G+@(e$GPhQp{%S|IVk$$+>`X)Bs||KxFN5HmM#4hd ztSujy>WDIg+BRRLXVLKbwyCT>k~G40LY=ukp5D@)0#9^zYwH+<*W*ogA)_;uYOv?@ zGYK+LN}HQ69AXhu+mZ0A{?gvWY^DE9Vk&MqUSTil{@f+dj61Tr4eXB8_UHbRpuXW(y=;Z zh}^W4V_-|M0$={;sc5mplhel_W`?C0=JR?PlbO%Y=k~HXBibqA>*G`TXAZW;+&&v$ z4_7VJYCfn)%tWXMNq9scRyGj+Xo@Eg2(MEvn#KhCjz_#CF`33P>grmOA8B`YNZgC>hb z-W0OCP_x6E_jww1OsG9^ao~N?OE0#S{$8=x2OBgkma{c25f~z15o7|#e1RglTC9y# zbk+)@&6n)yuf54HXAz`W_;u4`W@nVblA+~jFoF0*=SyO7ptP4oppivlgS~=D!i!X% ziGGIKk=2-xd3~u=KG9lz?hvuHx~KRMib29S!xS=UCwz_(;FrGD;StwY@($kqT{fue0Qb8a}A0R5zy%C(P%=4X& zL6cc}D%l;*m)hpDjJO(Vprnu0ii)dsn??;0Z&sS_S1`ypTfDr8D^vL^I5?GLqn8(H z_P^RyC>US-xFIpKOk<93q%C49jpyUmoHK_T;i=*A2k}icWev`=F^xoAq}e;4VPI%v z$>NpVZleaTpIeV5U@?IbLp>3XncLQkIDyt)@LH?VUDPY@z+)+1d&UrxKukjzZJSMS zhl~A{bav;WZnct#kFM1&W*NBB_Ju5?W^vhk^?NNth83$b^c<3GmmyHX(X@y>zSG^t zNf$Dxbr%?kspwQC*XG?&kgH9B92logTZ+1p@n3WP%6dCqwNhZYF& ztv_#AK}c*>Zc*TdiB2wL4RZsL+Z_&k_4NSdYW>6_=8ombP_aI?#VcK=t_%B_L@6FI zEA8hF;c&lGeBIrXAZ>Tp3n^^kT~vJCWY5sDJJO=58fWtoXZ#?|*7gdU@LrTyH(ync zvq>H|kwxJqf8HoDHCeJVz{aDL)w78i>YZN3?(^I~dgYd6b09~{NJgIbr`Jo7M^w|- zHA^O3`v?tRr#R`*YLCAZ0?U@>^jUK&p zhc;z!{9$wG&!-N5I!)1O|)tcv|*ID$NzNqd)v$5v;LCdg3zY(gr! z=15an^7LHijj1d)4y~QJWhCN7DcTi2zlz!Nfjj#xW~U5U$k*3*$fEUAuaS{KiM@y( zEwS^0mvkR*X+&=&rohfr?G@ zr?rg)QiP3a$uczCwo3XKB$o)OmCXL8Rn<11H=2+pD*o=@L;ZQME|W!!m0!EG{-SPO z&hN-S^4APtNvx?0DRu@{VoG%(^AY;3%LS8EBfm_;~ z7Als^muMMe%<;H^1S3Z>57&d<+)}v5ryW2o26*2)qkD?;VEUHuH~i2Q$@OZ zJMP|G_W+Zas`5BfRf<~AIECoMeK6kPL{dzFvj@5UE2aIM4!nNG`71K&Sg2}6Q`8x~ zt>VjyF5K#`?Acc>{K}?9kxJ=MNNp0hg%G~Hb6QfGNUv~bF#aoBJMNT!sUBZ%Y)zi< zl#^qr6H8@{U;xN7QMPQz$L5fal5DTdgYx~ zvZ`%O&%@y{ipgI17Kez#`z7P|t2iXvBb6f&@4ixG#H;2W%Sm?&9S@JFFFxEibj@Je z93n|^=#uZ_pZUBM9Fi0>Hw5x@yhzo~AcKfkJtiTm8v`6djd`VnBink5K^A!)b7iyE z{B9jDRixS~JIErp&mJMC&kS-1PSKWd=>1ubrIIF7-7gqsbY@p8y=8K&Ofx3fA3f~d zy^y1&bDg)`LY>$;Dg5O`%ZGeE|MZs#g1l@bxEw3F!@M-dyiiLg!8$%w2~)_dTRGH} z-R8(XDvbQFpOHY-ew~VYIbwNfr<+5zgLSLTbI1?4x9qGI2x4(AsBKI7Zp*ukVoJ2ivSdBY{AA z$zf_FV<+8NY;rqA`~A?1E>-oNwLUq4j7N2*?;&X-;QiiPdSa$ZyC6Uj$Uw>&U8fdG zdJZ+UsvM0sQI*D0q^CD%h%N0=TcU&P4us+`-RR|zoHY`+aDYR^qvE-#j9vn{(CaVg z<6>S@;500_n#gE@(q05{(q;(iv|PlolF!hPMb*|#J!DDxm6BmP zok6Br*YsSXQ>3!mg^l*)6y?ScqjP$0wxZcPzC9`vN&NXkT$gHRpg9$fi7#y!c(x^z zKc|;yOKI2`NyMpFE@0xFVXiA05g6_myKIBX`Fu7xkYQlS^aj&b99L>v-5yE$%cc&p zJ`qE*GsO7P>%Tz8Cs*ry7l^1?=}v>_WzR5n3a4xDb$8QgnBF_l4|BYD>une?_lzW& zplxP=MMnKa1I@P{zSU}$>pP1=&i|N)#jM@<4pE5G0dB}H(<>oE>54Y-WXsLaplqqY zQ}tVwTv8g6E%CoiL2wc)U4uE*@1m zOhO~B(BaM`!%`Sal%4R}^IrD&kH@8QfLeM5kZ}mY?ElObsNaDGi$=T{YWPFc-G?1yD{1 zF{*gw^k@X8wO6#pzL_=1!0VOP3RWjh7q~#gwodoa-^UQu)|q&Am(r4}r5{JPGPqKa za;KL`giSN5*ZIG}y=0jD4Q$D3>X5c;Ct|rStXL5^&%9})N><8SM1zI}z95F(5%~jh z(tegE#Vzl#=V;mZnI{n{Mf&S0N`EC}%jQjOV~4ab>gsKFCng_u3Jx~;$9<(usW3B5 zD`(^BTui)`t5@KVKxG37AExr>4b;(@Fxid5Asr%&5Dgao(*pI?Tlfj#%Ytc z%~!FL3d66)1=kIOAQlGpo{XLkOL}pqzLebPhss!N-ROgC%bmf4mD8OC2~Ewa=`q)p zY`Ql{XLh>W;(?J?!Ft&-GEQuY;Yz9ga%Ri2a<<6WnVKE1=vv1+{FOXe@(!aDgY^;- z-=T0Iu2EM6`}EudjM_F<($Cb`qCAjL4Yne4a}ZLg+WTh?v$}4~^RskZr%fQS@G}Nk zNqGHrcWED&K=LaeL&{;+=5%X=QKHsh+K`hX3swGd77;rU$mlk9iBxT#CMY)$e-+Dq zZM`gp>%{gJ4K=w`{AJH}7Ef9X3%|@I zQw@`F8X`1Js zua}ukZXu^5O|SR{hOG?q9f7eyHO15&pRQfWA0eVC7pM4Hf*6G~qGs4x#JCyDSj_J^ znyJ-6iY#ppw@&D|QnYw;ge*b|el`;*PQr<|ggtMQeTI2Z9Ej57?cx|t*PWt19uj`7 zNY5r?gFCRAVS5&ebUc~Ar=XA7g^l-ASkA}7YZ`RqhI3gHeyj1z~|4zC}A1#B7# z51c3CdTV!vI0Wfejw&90?J&(-myw(MZwfdF`GDa#m9=qx9-hE{vd#WfIU#KmCd zqH7>I(cc~-7?9&0P^^!6X`b9c$TJ-aAmDT)7fT8lU7~EOrHVtm z`-bJT7D~;R0>8xTy=%Q30vQdvxUg@C)#*gd*7?%Uv#o1=Fo^IfgS)&*c50mhQE_#l zibZmnJ&;h5eUEF~Rw{T{%%+wnL*m9tyN7vR`?fkHYnZ zM7+=caV(33YY$r}r#B|zY=fRVMk20VV4W@JLU^@~ooD>fm($0Ui5#Ze7p_$@TW77M zJ@GizwF(+JP-N^1Wa;S44w-7b9P&<|XJQb_we~^X)Onsw<((~WHG@_F6_^d;6XxCM zg-h$ORD6o8PL+O+)Ph20XvV3ga(P$gtmTQVVPRLw8WLMX^7-P#2fDQmSq)O#w%rTa zc8X#p7m^mTZ^zIii>`$^yKD)W)QzjzLG&Yl+={Ax9AEG zSP;iH=s6#H&(R4i70qdShn;SfrL{k0bC*<0N-EL`~w=yuXmmR!i?FA4RZ zs;U_TTst#8htZL03%!)3WoQ<~3M)SYmkotya*D*N`1f#eOiNgJZ46GViXqri6ca20 zPJD46qXUN`q19sXHGXRQ{w!D(aD|ZK@N<|~Byn`YdoTCw`{?p%I*5IH#!V?ZkPE9PWEB!0uXbFI^np|03>pm+N3N;pxGR1<&>`{8v83T@1SuHm7SINsWlQ3_Gf8 ziMsrC9i2`;j{BYX5n= zIj~d0-f?)vwdzab=6#(csFT2wmtOX*y-+n?u*_M5j=F*S%Bj@O*G* zProjX!QsIB>S)QZ7gKuh0Rc{0EQY^Ss!7g2q^|w>VxlQ=rlhVGc6gdje_g!i%tm=<9vJNU$FF}JA^}(Q=dF{R%MGTa?7izR zXT9-geJ|%yb7@Vsmw&ovUr+Mx@v-!FRu$^+aw#x5yz`yemUojHT6lW3jOwt8z&yy*1I~ZY981lziU3_P}nowF^UA&T~^X( zj*O9H-47AsaVx(>(b2lxRD&pTMq)3QlQPSAU-i(E+DaQ+qkRbxn0#V3kDcs0TH`*X zNK{wP7qB~Jn0dXRpSvhZ4;%u0O7G7Zs^dnHK6E(0$?z2m?1Uy6vnV_0!6S_@>;W%CQD9t zFVQ(ovF7wbnHX-}9O*=;rlUz%!On0K8H+bfmJYF#J@ZANG2o#H@D=DmA*$WVJO(Ed z)wXHuL@b-El?0n)ZN!eQLbZa66p6iKf*3i0n0@szZ(8NB6zSMqK7S^KMQ~vh zZ6JS?BrGge`Z5?@R8OFS(U~b$PUiix3%{BHC8un4Du0lf;6mQr;a&6WFi5sRKUCxR zHG6sE&2~nBe>Rr;Ix>(m0;)3{Z5}TITir1dGpScfbwp>fc6*4`g%?TOxea9TSVfY} zJ9~jZkj*X>7|8;Izp~k)03ntmb8l7gq(w_P-H>^;*D6hc5+jJSg7v&UCb|6&KoFYK zRQ{Y%a=TeM25}uHfc#*vsRc{Xx`o|FQl`qgGtBNfw~Ba~n*&^%3QmLpZx(}KlsCvBzK2k%SI)(eF|%TGn41Vx#8L@!W3xe~ zR`_!UlEl`)d7gK^U{q>0%oeeU&WLsp_n>I=XAJOCZ40?ZvTt}}fZRU+cI2wBFsXfY zzI2d%(BnrwdZv(NuZLq7s}paQ_6gn4XVo z+PYoZ={%WAyp-GB%Y$lp!@%jpp~c==eifq&^ZK(!$oS|fm?M#h z%CI&}gtKiM-=@of0H`c{Ary5g$@hINDv&iNU05I&z~5c7c(t6Ez0MbEd8v_l#eCU8 zMx@!{&pvqk#%3?~F5=zh%6#Lq>7oPm3$*6c)j8U>yO z`j-oD$32>U(z@BY8G=9ki!Z`ge=bbFsHL^^lC{{{PyXe~C@%2ZFaF1qzhC&<$U7I3 zj(zgUOP_><{N(GimB(HPQN6tD%{PI;|1n7GLqeYUR^^vPnvtKAOt=(B>Ka&?|V@`CzsRwHFxil1HNT z&IkDHI|g=wE4lbd4rE^aER75?TZ0Nu*N@CTYNh;A_h(MsWOfYD;AoY?1l6HjBNYiC zXPC_(T=`RaePTpSioq{BKnIMYoH~;T_>I~TOJ`5u+I8O`qi}1y#c? z1lGNhqpv3rT!n(RSu)20Zh&23@|iK$lvZJ&e#kOcpv)8I#Zdg(Y1~x z(1#Q7s6gTeB^$08)nwp~H*R+Ktec<9siaD(Lx zg++s95cpa@L*t6Rlm{f$g`_l?KnoaP5p5D!pIIHwy$yuqY^(2FoOJa)HL?W z`{7c0JX4;NuYXZ{h?$LDwBeEM4^9qg-mH+Jiwq(X?zq=arh>@QiSAiAPfW?s)h7rG z=j=!7$f&M@jBFt!B{;zMp6`;#sqkk>HcYBR}IvSn}a|%qV>A(b<7DO zI_Kw$juCOV>0+^Cp@@fM8&*n~5Es4Qj3lNmKkqyn$Hl1EH^F~mQ;nP4mE6S$?2dz9 zV${emfnvgtgwCuPJJsrpVAKedA20N{&jDCKYW<>pkR(!+8z4bZr*e%q3*~@AM@VUf z#fsUpAez-6-awv~Lvtpp#y9%taOj?ua)y*DFCF55oy2^{Soeo?K68MJbA@ zQhgI)5!EYAv-x^vm)N_I-w%RxwRI(jOTIc)Fj!AY#nd3a;V_RKvOIEFc;j339cXx) zzxWW*g=tqZdI7_PR;MXTNiY+*WB;`a_0DG>O)8Yn zY_!-MZXr~i*okqI9b^Z5bipRqW!BDD@DT4>Kcmx$7s;oJdYNFX@vMF3L=E+w(g$U$ zxdsSdgZg+53<6tV**z^1+d4DGYIKLhc?K&IFB==Ibz<_}h7*7Bp zOcj+W-GYURG_-c1R4si!`jib{+_nxgDGe!xJKo*Ppvj_BtW!C9^24-@UrGg2uQU)| z4;O21SFv!mtB<)LzM7VR*bC8r5c@zLThh8}&-iGo^!?ovwQuooz2R5*>HkLHQq?;? zFF$gk2QRqwF{5otsAEY*OXHyTX^2ilIh)bXkxthd=5viW73fgu`MpU%|JIKYQ>hNk z-XG#mC=N1TI+i%)__yb5KLY!^0PHg$-9SP&3O@S_z3$&*#=`@pqeRrM`DBqJTWa+wN=yyn8d%I_WC1haU81y_x}LVk z|2g{PLh(^jn$Mp<%tIx&lEFHcdE`1T`@eyG!7zsyf%i#FD^G#FdG>u|>4t%efd)p9 zK_y$knHj5~bFKi7JwgIY+*4q`P{B!*)>srlV2@Wlif1GQfqmFhV2@dXzakuKXV$DM3zP7cvDZcVEoqw)Xb^(^4^`Zmiyc>UJ~9uBl+s9!_^-;v{>ga(3UGd!kp z^Gg|;7F^hBAd3lxB2T`yR;*abH_}|ng_7<8uyfrS;>yqz350_|kXwc5+HBq+6DLxM zuLuobAc@jWo;$2alp%okgmhR~XMDLvJaFW(bNhP1reLEg?!oFS!;sqHjqX0W0uuBT zrD;m=R;1eVZ7;k7k+0@JekAg&l!Ak@34>*Qm^I--YZVhia7@&dl3tEX-?lMo7H?c2 zxx!}_N{lzrt^5&95*`V?Uo+k-^>u=ih*Mqzs%40w^ z!=_Cj3)8^s$$whS>Sb#d+oQHWj{URQ@fb=$L|}cakYUswE-1j1s~dwXQl@g;bXrSt z)x_Pdk`rCtM@PE~+seT-lA_KBP1)hkAHA|t%EM9Nijr(wxvWG;0B2?kk0y!Y$M=np zWFpJV+XUOz1snzm1<&l5&FF<(kg8ZI0@s6iFuiFJA9m4_7A^)?jh<)Q<0$Dl2M zo`rl^{ZN%@W26HdMoQ3`f#p8$hAW`g2C5*OJN&uBbS^Q?=P_-7>aprC7$Mgn+x+}) zE)XBcg$sswP~H0S!Q6zc&H$_cRtM@kddHsuY1vjbO;%H^oUG)f!4ybLrHUK2jT&i~ zN--;B0xvZy7u=g*|CU^UiF_z#o5%4{#8m1Tp-r~ zO(Wwm^Fkqo>Gfzo45z!r{Xkz-r0p#3AB)y05x>qub6Wpupb9HnY}UsQ_(4tmc<;NHR23vBv01 z7U&)`kv*O|Fgn@1GouNqHH;OwZgvzwiq(08VE@3v$KClO1Zlgx_5eUhX}f1z|K4?L zy+(>q#M{|%kOPYt4)9U1kmAci`f`E=7UliZqpxRF zDx~Jd7!E}K7lVVW3_cPyfg6=CnEKCvZh5RxE@;sz+;F5eP3d=Sux?&#SB zS?sDMx3__?D0-onPISI>Exj?RcgdMCz=1q(CQu=lwp%1%&-1A(z(1!l0D&9D)hIU@ zkcxj7DCs48H-}jr;W^#i#txwf)+MrUqnt(dS6eNRjV0jl`b#@Zl8y4+Ameqgix}-V z)H9#2C*xPY8Epv@{H3(p05XkyVw=!19eyce6mYK4K(QQ zs8+KEc#&R9NuPW|nq1(kAf+yYC z0-v9KkUbf>+b`^eYpq=yA>ucO6ySN@CD~wgq*4`&8Qhjo!`FbTy*_?9Lt`smsbF-W zJ6w#S1F45_*SDUdYr@%7iiJvE>Vw*i9EKeif!9}qQ33pcqWC(vXCja`@&_7;ErOi^ zqDv%K!Xny{`k=V&dm~x2XTF?GMz_wzaY#a}__50BEg2<4^fS$tH;`@aDrUrkGFS+^ zBBE;FHfy(G6;G~R&oB~g-l@FNMA;@VwO@8-svMKTenk?NzC8q_Xr)TgrMiAQpOJ`H ztrsx~cSQ>Tr9AUhtS+3$Nz{D z?MsCq!HH6?6&O1(?M@#9KP&cbF!43v-kt8zR=wkU11S}N@H#(_tO92T8M9o=f0c~x zSu-Y}#Zw|scB2n9bfo>DS~E1`eU2x%LghBkW?CryJS|zSn5tj^jl$BWa=IHkob54| z5-8M3X!Q!XBir}5OZqqot~+_%Iwn~p&{{IOwQRerW^O8Xm<8THf%0(!;QF3_;QFAN z#R&SF{pIyUuo?e|>owniS(b!%Z1=NXUkV70+Hq9JN(F=_!@ZsZTyRmgZL&%DbkFHQ zPNH*}`u!lYt5C8%iiy2&1Jn~&e)+I%;!lFhwpr2m+gdMt(VuT z@+^RfphM>S9u`2%yMCZqCYssSav-RAZ;g^trU$-G)0OaIRFv z_6Pub5FQEtF27$g%$QEmZqdkCidebRJkzhUL7MndVBIz*wNo1&#xXmc$ps4^CoBpa z3$YD&Uz;Vjj|oEKLN0J%)ElVe+=PU5)7WC0D(z+v64ExcKeG${3M2e11Qvx{sx>BNb)~$y?$qu7R2?%aXgu?r|-62$eBF7?*0Yl66*^JVzlN*pg03I8sbIBYi&r6S17@&~{It`=*puM%5D2-ONR zlITJN;;WmZ_?nc7$DB?ndMazA3){Ltb490^-v-AlVzT58G*65D`~t8ALP)UALUePv zVK4M%to9Lw3nvP(r(h^p4C$?ppCV_iyQH4%)B4ZEfpBLzCN~6Llf^pRCE7ZubOio#4j#t79$3SY zr(}fW5?!)fNLug66wU20WM;{uLvm>pI43UvH<)^bmnu;Da}0D~#}Uif*NO(^ghYHa z7~?Jw$Ue~Xr)xd)8U5_8b(tzq&~vDch2P}hYQPjc%q1>(oEMZE!_3Z9yhy)Qcr+nZ zM0pg;=|sHt^Q=ynPqH=&F!txBpltv+hvopi1vN{y2k7-dW_)pnHM#dipJ@L+zyr3K zmCOTdVk+LW2HtA8QF==`nC)z}ZRfuIT6U=WrllAcbHP~ zcip3A$Hu7T`V>Ei^Jo1*tA{AYS2WC=blOgAT_Ck$DfdPa?qd3@*7MENdjZ`;Pz7(N zWaBfu?|R@hI9-LiXoQF=zZrd9FS%dQKue?s&K-sH*|7mx?w~G^UZ8Il3Y3nV-o_4q z`Qk@!%TR}}@DM#{yNj!6wuOUaa1-@9>*&N(Q5a=S$DSZ%LQxpV?`3{z7PpzE@{Q#7 zlT#&7KmynSsXs00CzoC9D{o8@Y zX~X^ue$h~8OX76TxtNj&Jfih~%t-GeVw746BRO22+f6?%R#?u&ff6XsfV}eES$-^d z0xuONAvi|Skn95+K_4&IEU2@(iO4fIv`k`+KyrV86j@}e{raz&StSY4)#|pHZavM} zF1XBpjnE>ZTmv66W3Lb@&Ua@8F~konrx`dT>PiMYzR^n*iuQk8XCNF0=Z#oO(Atz^ z%n9gU$O3mUXo+HV&Yz!O%E?IAIzD40(6m;0&Y!otdQLv6Rr0(8yj!DVW#q3U%mZ;WG9sTjO>_rf!4#w5=fbKI!%v2%J{YDj&beJe~BzQ`f|SN z@)gt7+3@dA$*zlRpSBT&QM) zQiVbR$H+)GOX;xF;M<=5c7L}KyZj5^2NhKE@6o9=<*WcBCKKNq`{>Aw?9>_^A#iMS z%xu^IhC6P{P~IVdnG$*A66S* zScjL&3U*z+c%kk3fSSC-Gge_MJbm%om$qG0Pf2smLh)O&70%uNSv0^aSmWi~8z$kn znD*~)mv0b;ZQ9MQ(C@f~v%m4t7*&O4hws8SP_V?x+io``Kens`631&zfFje9&E6m4 zAr8g&l(I>ggLCT?a8L`xnuCnwU0Yco1)9B+#rxWo8-0xl()K8gn_o1nAQz_J&kH)5 zs7kOFBBpOgaqZz(?_bzxzOrNBrRh!EMuKe6e}N>^k7e~7Bdbl@y^T_vemxt280CHu zFU^_(pi@*Uub>}?PM{*F1+na_V_E%-z0iuN1;c=Qq4Lk*YT`xMEwjARt4o!A7M^s*S_VJ7I}2x2j8 z_i{!dH=hi;Ag=-h(own7PUV{IAL(mjN-JydC znv6G143g2MnNd=!`5OP@LtFwZeiGA+mS@{4GB8Nen%YN4gUeY%K-kAkjPzOUc{;WR zi@!I-B?IkP1*ziR?XtJAaEBAjd0-`*${S$fy*q!sd&g43=|c4@Wp7JG4)A|>VPRlQ zex>otlz`|6ohO|J6E5U65N-*W`P$hS&h*mC*|#{I$%3Bz5&Z5AsLR_P*U8E4P8Ie(l~! z7nsOpD01mx47@oEksIa<4>P}n>}Ko`%t>XX8!2($OR^Q}Ar* z^En1kvyc?;1~ihbu4&mxz1gevWOO^v*oiD{@UX3dLAVyT0jzgT2f?~M^7|3Ze}i>z z5UkVg2EjTs`27gh)dnA29B}X_sqRdH+ZPW(=HkW(WO|G?3!e`%$2UUW6WODbaKpv` zn3UJ-IZO>PQt;^Qsr(TrfG`SB_?te@{i43c4m^5iKD~-uxViz-p&k^alr~Ti$)fl` z-be@P&KSYBl??FG(C`?R9Z$Vd!NQ5mZGPxflZm{h`4SGy^Kk82cQ@SJz`0n3T?kfl zN7fLlBN8m)PzMDnj^kAlH}tfE-xa(MhRuE!KH8#N%_E z#GQR0n;1R$#Qb-1!O>;YHr^;$9KuhBMd)p(Xl z1{eu0j3Hi!O<-!}Cb&GPo`-L-Nzw<7zyVeQG?DrF(1F=03$IrA;Jv8n8$DSJ;AWVo z){eu%0BxZ6^sVk{g{7(vtV+e++ZjB3vPwDs%lEwY3NZTm%exv&IZ}?Sc)lXCc%8I- zd(?j(T_+=tU2V4SuCc#dy~dN+j4y1GzZt#!#J?q5f`1k+(&GgMLe*5fc0Oa|<{9E%cp1E|YcJDjz=qJC27Wg+qLqgtE zegY3eLjbiurR`t9Bly}A8u|`!_ZN0ObLqz?LY_JG?EkC3J^s?y5h1jQFj`1RL_*w) zg}XkXg@E&|E+mAe$a-ZlETrxy`|1+*g@nlCZtXjH>YWo`eENsi6JC3+>C=|!XThxTvdm?9u@}Ms0)e2rt+gsf5_iUUJoTnlw%pHP{(hDAEiA?m%fz3yZD35 zS}9v$cd5%YBPyA`Dq1Z2+YxfgrmS{ijdS-7Pg25cOe>zsAG%4MuUZz%!iMh;aPj)G zhR$}eb)Dw4nStcEuA|S}!IKGz)od9TtCf{mAqQr6+6diD%~k6w$)Q?k7lu{8U2tgw zWIWuApMNF}R2`)D;%rdY1HT=LMEylLAlKL^R#4lV?Zrw6L*GP3kZ>;;H8rP23&plFv)iDL85JVp_n1?m2qTZ z>vzLYxkSUM2q=n9Tp>k#)l$I8mI-`SsN7c=WAZ`oF9z2}2Lu#VU0ur4GVFLHUTpAZ zeGVx3&C}g_NXK<{@S%lOPZhB{)8y`gL2g>gLOD6wexwsevf-<_QCA?bOQS*o9)Te7 z2G9hVvS`ZS$2?03x90l&Au?k9-iS}}zcsD=dt%zp0q-#qq`d+OVAcAL#bs!+Klf9o zjWWC-ilu{VClb-!=c0zczty=sY;A&FSXYOMd-K@PPm26Lzx&gFk(kDB_|gHXza)J> zjG)a2Yr%z_whzD4{qbQZ_U!<`@c(XyKQX?eFkc0O9HUDVvvS%^0r!HqOkZ z)4sI|2go1Az^Zmd4xTYQDmG9!c!1*(;rV}czVIqyeoo|YXEDe~?NWg@cyOv*ZKDh= zwq!wFdPOl?9K9FfwCMS%_rhj88K8jU@T;@X)+a^7=Wg`TzY=!80u8fRyw+rTc!;HO z=>bz1I2ylUvXy$Rg4uaj4?W?-Bz&k2q!lRsHGYUl5jfR#=n1?j3YE;Ci-nIRAlj<6 zo5LKkxXlht+cknVZ%H?xi-`q3Siw@4Yd!9=1YA3Ea>Ed`0aC@Nx^JoJ9xihIkONnp~%5v`tQtckAeT z6tIA^ZzhYOB@uc{^bJ!LOuJ0ER1Xl!SI*MFMm_UFJxkLHj2zTjjSLzv{rMn@;Pn?b zhPYCJM93w%P=PW~3jRf<{Q#r$BXxNT$d-2KkM4r-?t$WXK+%zA3gi#ecO`qlN)8gP zH48ig;M#IP2P|^pzOcWEAQNAm06jPv;Cd0Lng4+6vAe_o*Wr9@wM#kLpx?bY@+n-e zNl{FGe1K_3y+6YBERZ~F9UI^t(?O$QJq`nZ%6-u>QYvU@H!*=>V+3#UgHod*xuUQ8 z;hefK{k>QQX(`VDQOvCM7srFC(z=q-a{ye6;K!QcL;tP0&7CvEY(=QvC-0~pTjy%u zVv=!R0YFo3gG`5tdxWeRXoug(Jjf(UBgHF?_-IognfR}%YH4e>vz!4YI+$wbpbrKH zjJ$0zOV5=3fass|$%~)EBhOp-x&6FUi}uc_4D(p?4bUf*BDph~h!35ySF&Vp1l!@? zIM4n4nDE0Ua!rclM$i6#l}63=T;P{7PgUZgia9TN3PAOn%42l311$PsUq79Mr9g)^ z3%Y%ZwQEf214^}cd;-+5WydB=bv3lZFGl3gXI*;Q4uAYdx}M-_6L`-hMU$?YzvuEz zV(IIL(@y{opFRn#7$n!bvyhwrqMR*VPPw1^IZiy50k%cNF;$|6Hk1hbtUBX!uvfJC zA(cc@3?NN${y0_2fvWoYcqd8`FO@e&0BbBAb@R;YkW<<`H$c|u4Tt7X9unGiqaPn; zw$5<~i#RMw`GGSB)`8{SmZEO3;6z4&YZp2~J%S^RnnWRb3LQ zh|KMCM{gGSi+bWX8fb@K<4Hx{nUO@yJeXztaBT!K@NlujUfT2C({^}tb^h$WAKT$G zAk_%A!#6@Zym-8C$Yff^erYOywCs*EL_f(@SoT zS`lg)E(e$CcJXb2Y)tTgl(u_%Vj**1PjqO`C;^Nhe^>*6T{zT6*3FhHx}eBV zEC>g=(AyriYT=iRbjk#e4gSdg4`4S{&>P1L<3D!9f9EUt4<7*bU`PBv0Q-M;#6LXL zNW+31@vQ*ZlYlOAWo}D8^suFh)LW(C((MJnKJ#No{C2P-e$G-!ko`XZJ0l3#GrFGw zcIb$2a^8TBcnJXZPnW|1uoFM9(R5uY)=hj*39wFBXXJ$BTE(WW8E^Ima^WKzKv~Ti zC6Ke`1(algI2!rIALhZa4E?4V+JsZDut{kGcNqgBdbHttPLz}+9^{`n0L492ai)dy z+?M5~*`kJI=!l194y9Ow9r3QG9q~!Y!H)Q!;tE&u`dI{W!4fRzDJ|T+Ys^;V!%nrk zC=ou)!CjWP(6GL57@A?D#CQB9bS`wJRA1Z~hOyJO*2gBO;PZe07E$hjN(^MtUj$+& zfMJ*rBq4Z>Bs+T6By`pk^~AKmN4-oHuxKtsrG!3pFm`2EuT(YyR(Dj=n+QNv<1hLE z0(4>miMwoHPz^`GqF~RYKxoCGW96XH2CFhK6}HQVA0F*=)gWQv_XfGrUW;L}Xpk{s z)2nlIT$hOAt^feAL+7Qft`szqB1N9HQP7nkjzDBYx2ouk#8iO+?Da0QX$G36@`G!r zFFm)vL8jj-(Xl$i#E%Iwv|yv34$ZO-_jz8LIq(}?P5cBHHbJM2tNdSmoeenC`~UyX zsT9pk-;^PeVze`}Gjo&N6o-9}~hMZS^@A(RqNUxy-qRh6l`^NeDPNUjvrMc4mB035S&kEMC2Pi{}Qa z0l_#Imcr`X7oHdx=Zo!U-is%(=e`;lz@7CeRLlsWhhsI}I=ROi$)tia#0M-&J}`R1 zjPx^+=`h$l9I?9UGoTTq3d>e~4Pjlt5vDxA0A9O7LC=Rd_;bW#uxo=POt121pF_Ov zjDbK{`(h)=<)E6=8_PleJ>T&$9>$T3W(esUi;hl5^Nl4z;H87bEyL1>J=ev9>6B?Y zd&&1<_+l+b4HZ7M0Obvt=SX+I8Tx;sR`|nt3DG;FOkwG4{E1i2G9Yle2`?{k%llRt zdqj#6+j&_V;+J3j0fANf0c3BdRsmxVB%gqTN_xQ-%X~jfdrBk%sQ^IwJ_PQ%5GCa9 z^(p~NH}F{fDBe_!ztwoGn%{A(0fWt@g1nm<&PaS4>y<8L8%VHC`) zZSJYG!m`~96FOrs{e_RDKfB1KM|gSv!`MO5_kcB@Q@+aBOCdhJ%GjG%8GBJ#Mp{Hg zm9L9u)k3&^{%S^84N?Z;kVoKhF9WM(Shd;*IqpeKAlyOV0rL_GJuiFo643BvQ{ zUXaOvM0}12UO*_G`Id+WOGWu{HR#HKWQr%8;}4yKRw`;au4-it`OOQETyz)=wr2mJ z7gEbHsLB_vCgKa<8(>NS}v69m5d*eK-rIPk0?+47*_NbKlsZ`HLC1e0Rc%KawG%`!Rbj z_yH#H{R6y=#rf|j)!c_c@8h_}v$RKEC$Na!Q;)sxv*4U;!0fJTGQ*0-0J3vBL>K>o z>^Uq`-takF0Gt5X{ZNvURme{HkntHdOf}$;41nyQ$%WN=-Nk@@(gKhj)~A0Udj>%E z?9SsK&ff>fes1TRZ;(Ce(OuW$0NK;uMb-NkGq|&XVUU#{pK2^eoUfbD#{9vQOZ%A_ z`Z?yr1NIN^+ha~hRv|m~KakzCu)7obl#E#b+3&akWcPju@<@n%pZg86 zzxoeke}_1q4s}o`T#xh5e~xOnpLtk(mH^Qj{X5+um8!e1xmRWJJhU<$R=dxz!Bv&C z+#<=&<%%bMBD>ijEB_*cxrI7Eq*HryAFc7eQO~k(aqrtDD{rK{Fop!Df7po!Opoq` zT2c?S%6|&E8s#i@ULieTeapnZ{ZA&ITv3%{^go$+{co9gEV8N=Qp>9yA<>Whub#h_ zFNV}crn~;N^zl^#BQt1hkY2gBh_yuZeTZrWHzWA@u*W|kq~IEPz%H4LsD_%b^p?M@ zUuKqt(H~{`xD`T%=RogUVXQZXSva`R0PQg1z|+>S$$8Aw__((6nyBFVfh{7D zI895(q1FQYZF39W&`g0?jiacE2-9I;6cvIs{)s>0Q9J#SkG=jca0~25yzQy-0^NkA zpWage9RaJ(B6ZeRPCbFd|2T~9IZkdrH5xad3Q zz!Uxq@U`}fa*=2p?`;4LAB*+=qB&tG9*cB#Sc*RZrRphOw3=bvuKAHS4+zduAst2Z*}g~|(P037@ZQ@d$%H^4Peg}Pg$ zL5e^luI0nQh*kT<;I#{~7jE{onqf>=@8qo8n;(M>5i-Fff3XX`IHtqZY0T znP^xE%ig~MmjLpIrLZSH7-Lot?;Wi+%NuzvW2Iu16bJF9;9Lj-y`Z5S)ZAqVnIWK+ zP8?NH@z|Sf0*3OadXO8($7UlN%KMyKxnc?oC3-S4=mI|g(RL1MhhrI9vGU)9ow(0T z?~Db5x%+#t@~b&-5r$OMsQeOnFdsO!2ZhkcSI&m< z3|hDHGAk7z0~OP(H5lzQ_UGuay>XyX+I$ zt%@8lZ6X*6U%mXncKDQS0%=Onz$e)yvs5UtIpMy7g?Du?9-N^w(_v4$8Xls`G8lu? z@$eMD(Ec-~KRjWP?+>QpWpaaCD@ZlC8i=8-ipQx%Aa_ADlV}7k$wy7`F8v!7$HCuK z?d>K4JK3te7}>H)*HC=CHmNB3-4gZtZI`syAwzQQwhfE42W z{@O`$!+%5ZEiX@7`m2?QPrG$ZlffH-KxEjsBCEKou%mmDw%XG{DSQ8t3vOrAL1|P) z%_@;;aP|BFoH4Ig_$uK`3E@?->Lr&q9DPqQaZGMhJ}yM9iUqYD}L)DVX*$S5xeuK8W^1QasZk9eh5=5qrSS zcR*iuQiNqzS?)3-w6>22D>*w|3Gq@p26jg{u|w0y)fi0QzST&40?*5cQphL)(FCKm z)+rszA25B+h#gQOaJ#FDgkc`Vd#O1j2+1UbT{j06TG~l*$;4j!rL}#ch6Et`F5*xx z1#*H8`JH4wsl&zj82*4mNm^l0h9-yT9h!k}wjGY=7XBUn(^GF*e9+hMx)rqZREg11 zKfC02?P-7Yu=SlTu?Bw&`dEV5ed)b|_`oaIPgqFhkjQ6$IQ;gk1uW$VqaUM4QJ@^G z{+z+yT;qeO@qr=jNT$z0+wYxcN{qMaPr0oTp+&neyq=zJ-YR&udQ}^jp*(MpV7=w6 z{;vhscQcHOO0e^@#8rxHq>!n4J$>8*i*`=DHj!rO$5KVex7^2I5SSEib>QZvL|$tk zpl0CAX=!O}hjoi+c58I*6GBgGMo>`^hDYRyb{Q#F`~%E+?OBiTO(3Jr^Z&>1ai+FBK|BBD8p>2N zB68tzfuwW3eU=(L__uGZObk$WBNx2Lh4>A`fBX0!{?2PHVWHruwla+`ImH^td!Dx+ zI_j69KT#$o!*jr3bencY><;lIdblIx26L9b{RJ2=+_xWBVg2eqM5flK~YNqWz}8lj>f^cI>LH zGsi5B_`Og6Tw=T>A?BWImA(wg$UoUEvUkVu?(_7E$cl|<@AOX| z`fY{14soT<-1HAez3){yg|7R$h7E>^nY)`&6P|g_SDb6t3HRG}Z>ibx)bb1Svs+(d ztvJ5-D>LuBujdCJEVFL8KWM4nq50ns%5cu3`(QxEmc|RGNEfkXQ#pu%Cl8PPTtXJj z8;8yNc@Fv_9YA*Xq|W$2nOck%)%9ZPQSzFI81Q~x0A{*uKFe18q@C#IS4aYaF`d<%R$-K z&mX?zhir-Q4=$Vq7aw!}i)C?d;{W8>i#}ZYC&%u-nq&8V(9gVB2MH(1$pK&*{k_eT zz{^BD@kufm>cH8>9joFzbcQJ|&OcZof5h5&HOKB}%!2Addi@<_RUKGwecS*mq?&3k!E9kGDY39u?MQs!mW&8qLD&G+h5Iz`xQckkZA@3C?f%=z=82exZ^$W zY>%vl6z_j?>~3lw{wK#Cs`O8e-Jh-M&<#yoqSim(0}VEIZaGWLl;n5DxQUK-@Qa(r zLwH%2FCh36UG{hMc%OKGkTUB;V^fgQSa1)T2T z?|XrT-tYgVd0O)o(_^q@1=?s^^PybL-J_eFV+7;+RO5rb53Pe^)!9hs4D}PuErPcm zq5m0plJj6>oj($gzqheL&}Wb-#_w7J2o6TP9}lUt@_iK!JM|@~J7m7^g0CI27fP!h zir&wL{z&YfYwmaqVEKn`aTxSi_Q7~N|HPlo%$TUoF&@uv&xR`#8UQNzSOj&JnAv(C zYI`}Z>~Kc&beO+{^S5PhoF6e0tXHvjA&dh!-Ot!L9cB;c7zSf3WFoPB;Sf1B_3Jx# zk+;kPg{Y@oy}a&}_kGCd=~X9nKqYWq9Qepv$7Z7&A-^>C zF8XQnSOcct-J>baHJCZ~x7w$SFJ?x4W%u4j!u`=Wye{G?yYMy`^^9*rt(oZj$EaM- zx_IO9_OZrSlJjrDyJ1vZG8gv9za3IeNJPMN{9VivM5dXbO@i4Et@7?DLLpl}UvdokoBMax(P7}W(1jUn>H?UZ_tk&m zjxpTN^IyjLtA#IJ*bB|2;SSx4ARIXlez7ix7*nkO+rF?T*`0_%(B&VN9bF(9-=6n< zfb8vVBY~ch-v$@5>D%7uipEaoDkxE_FLn;S-yQ+z8_64vyMuu$Kwbz~j#ur#u=);4 zVH*-NCZVe4L9vOd{v&ZQSdSlj)GwNhRsd+SnEL)kmGgq83m<7=^kw@=*i z;*R@y9$iK{%;uC0Hi#ztn1AFTIqh>#uP;78dVIPsdprFzgA)taa;pq#`Tl60Yd{Jw z`s@Gc*9vzihLi^H(eu*SA|_rOe*zTWF8+z4aK!w4MFU(wkWGSCO6OXT#Gi2QyMeME z2X8S5e8d!o56(u_0sd*0f`Q|{aUmle`U3C3r(`V00DB7y^H(+Hf_nu3;LgrCun^qq zX}k|Z#(PMTfVcd^&c!-kBQ+V%2DYM>#m3^&B{riks237bl{EirH2qBb_58ZY`-#ff@t6gcy?qi!aE6A4|bF#a<+rwk)R)!u>`CmiSHp`(4_Z#LN zKIrxT1?oTkpGo(CdxZoS7Z+4g&adBn7qUJj^}8N0z+&vtxxjjV3v=rU>iPdXYug89Y4nq6?FdF-??Q2o+ z#KnLwirPANSSkzE7i|MX<_UjJ+B#F#m$z?@oUHwF?-#w{+FctI`C(3?N1Rhh?FNxO zN7aHHPit&OkBQB;@ll;=_>N7_j-{u2t>64o(-a%k%U{bY->9a&Nm_6$TiNSSM%dxJ zb(;fN-?<3zf#!PYV}p-F+Qdi8e?f)vkKV(RCq{p4KOznkDoOd25Jr7hZ&Io}fn0V} zbB9t=l14WDEN(xwBX^oAEa9cRF+q<_O<$EuHhQJ9-CQ?~-R$C*ldfTHt%15lBTY%p zQHIRSWCN#Vk{1SyVN*(U;MGX=h()3huVs!Cm@5f+A;KoPZCi9}74=fo4EI-`_BPYg za7L1Q6BKJBMzVwDyzsB{P%hqCsE!0;N~$A5!*;*9-avp8Kja91XR4B!UMQ<*?Ajxo zOTtI)_jZYRFZr;8WGN^A^m><#s+6EgJjFXJI@b8)e9eob9Ow34ZunM-r>49->NV?t zzCA{i`ul7|XaVh|Vhy23ayu_tbQyiH7S(wIkCv5Dr-dx4EYgc}L5WmxB+20gM2}#7)npB!uz8`?cYuv*E7(myYhQT=Rp>zT07qW2&4W zP|TO;>Ix=$NrZ^r?CMiR8`uXUVssJ|)>&UWz9nlNCrkU+gQ9hrfvhu17Ljr;%InFu zbbVBa8+rEEF`dUcC!>KK05ga+dIda}{K0*5Fr8e0NZ&@z-$ z+Ad;3%g&C(9m)Ls(^-K-Nn^veaiKEtkF_|u=KA(J+B}ifC2JfMXxq9ywKWp0cE3h2=f5( z)yq?uVR2p_>cHvb$NFt+$_KR=n$}#54;?26=0_0rh6swuH?mDARw}6B2Cimejxfh& zFR7s#H$HR~)n1`t=2B2ye#1Vo@^Pe}^Q{H4gc{_`qa8^WFoc>ECrL7|MY}@#?q14f zjb!CiGsCD+PfGWxlazhGO+`#(<8N+!f@+({lOvjj(XDWSwU3*%?wY#VC@FFxg7S)k z+}7NT7RWZ8#w*~sas%5!G|hlEthFNB~GI_g;MQ1LC) z!3k$Bc6he3*wu9i<$7(?K-npmH{4A#bJ|syW8x!C_62iW-G(E7HE`Iz^@uayaJgei zk{~41l=|d-Fq=kf8X2{tXeq4w9=|SOeWx!U-_vx#Q4>dG?J^=hLigHUo|LQUAbhVZ zPr{MvM*}O99B(kx==ILxBY&DlFj71 z9vz3<^>qmyEqP{)<}-Jt?0tWHnwnBOP z&J8#Bg{(bD;+#~_Q5wFdPTr&Y>$8|~@|s)CeD(cq$~van)zR~z8`Et+px zD6UK4hHNvO_uxwuZK^cBKY3G=i=$hhU-MZ7hr}b%LQWoos;WEm*xoVujN8ZD4L2RH zN+Qxfx@lNHR*#iGl#aNPoOB|uD1x9T{%&KHqB_>jNvynW3zc9IA0I7c6%+T6doEoJ z^R4~h>CpLGg%(>^Ca8YUANk^Hf~F-!I6oNIIa^Afp!*n})^5D5*Ou_p%T494G#l8}K{pX-o#2dTnr#kFgCmsl)pZmJs2qi&s|@q1TI z^AO+2Te7^PB}KpWpnjcXl=>vtZydf-lAJ%?nUY-ek-OI-tcG^F>3vAqu{6TSz6^`u z??lH~DCJ?kMaU7f8g;X^X2y=`nuF9(iCIfhzyM$F=*elp2LYe=GG}m)*~01Rf+Nzk z>q#j)D+1F{qiK?J#2ZX=l#^&@OSTy~a6tW1Al5FrCwYIfc8<=?RFzzk!J3cTU)N*z zM;+aUrw@p$|13FhK*>+Jb5rQLF@br3B-n*yg`4I^b+H^HuT>M8HomrsB;b1|bh9+j z7KCZxEPsQF{jQ(v7(#x`7B|PD!JJJ|oCJ!WrlH`|&h5D_+8f?m>}wj`F*3G?u+k*W;-Yi{(BQFBK7cB|YydR=2H>N+kpbh7ncf~Jl@Y(RN+ zH>Xx>Q`bDFcrmN!)$5Qz98$vz(mO;ohZJ!HM50;5k-Vg-0o=-T?WCvu6=Ac3b~Y*~ z{Rqcg?**B=m(tdCY@=%Yw#S(LLr00jx0(_Oc3dm8nW2R>%4=jkakyeuN+D`RN8_SU z!lVsIc0W)Y%~v`Mp* zU9{CLL!7b%L!-s|sW%g?cHjhN8}zTM64dm(Ca>xb-;yRJ3abp8?A-RvO*|uBrD!#p z;)kcFCPvrcX&Po8qqN8r7R55m)G$J9Vqg8>_a>E_&r@zHYj3+~gIly0Cu}&g;YEs- zDTS@cqR~cvn_5wiw=^3|YJHvby9*&QASzx%3#r5on9k$9dQp7yGF_aVd75NvZQ`$_ zldNqKF}_q9eVAvhy%pso&3x%+r>?HCKF4HRBM-Y`YkjxQrOr|_L($AEMiaNP{tb&~ zf3@It*UA89f^U%Yz|WT-RY8lIn3SoIZoUv4$P)@@qC8mmKY|pv7Fv(CXD6Zta3gv} zb{f@sJxl3FBuYaBY8x&vZ)brZ`3%DwZ|K4=(n{GCJUcO6`J%g3LSb6z%G^hbwE25% z==g;N!6#*T<=l-d-CDNW!(d0E8hIift!+usF$=?+El#9|w8d}H&^9-8nVJ#vWuhHP zXKn~_RwiMcXs_(%tLduGiOg!68pf0!=Q}jAJTob;&}bSdU`Z0%_Mb`~u^?;bS~qSe zykTWAL+!CuktZWvvM|vrA=Z)6%YyatQ6cpL$4HKv1*I-PBLjV;Ad$SEa-d^)Q&1u+ zLGhk+O9ZiLZStvoM>WdsErwaw>ZsgM!_a$WlB7J9wT^c;nV6~4r2NG$1*LS&tr)t8 zb(pw}FQ-wZzN7X?eWrqSdelIZ3{^zZ&q9ZmNIx7?QcRl4Qn6lRmF0I-eJMnB!!A*F4ZhzJo^Y=aT6}^@~sZ?)I{3k zcT}6r*iBuZ3rxWBVWBITq;&eFst{EKfwRAPq40z@5w0>%X0EKgsRP$hZ*b-(X?e32 zL#xPJ71gFk%$|KL5Ud~2Z968|W!%CviR$FYcO=%zAJHW&FZZThbv=KOn3R?!R#@xk)Oa{?e1AR7F`>q^&H@c`YSl?raU|x=L0lu(vl4 z&2&D1Nj|Nx7aLF-9XAjk5V22tQ`9m^#C}GWrXhB{a}t_gp3k-$6O3?^6Tcrq(85O@ zle~_0x~%Ks@bkxB-t>mMK*$&+v_44zPd)O*$S$axh$OSn2cOYxlS+QPY*)Trc~Cv} zl7c0s6An{$Xy!NxY3r)>#ELwdx54H0ttnibwCtzLi*ZphLBZ+Qxc!bFeJr$9DbkVw z>O$9wW{CZ%wFZIIdr1`128;sr`-+LZ)FzQx;Dt!(o!sQKY|1--Qe)0m!+A40C185C zp7lJ>ED?7)xqHp47o9#8GeXIbQilZ{(_w=%CyeJwM6~juj2LAX9^y&X#dw@pCbIiu zEJ#q?f5jxx)J(QGb;-~?aw?*8bgkYOvrA^f5>~gXsF;DswY1&8jCxF+5}TbKw64}s zvWPcZ?42uyisB z=}J_!`p&9t+>Ld6@kDzgw-FV4x8#?(Vsrb6cr8UOZ!WbXPm0?3Bw`DC-$d=vKm!NS ztsvZV&Ec6_UU1`ib-7k)qjo07DNQk@n!X^eeEv#4nh;Obw2Uy_k#^D{<#FIZo1pkA zJF_+EX2dVF>5;O)$pU4awawFPL6Pg=W*2SiY?^PEAUIfitCf~|%=(V?uG&aL+8*?D z+hUxBB~>$kystqkRaoef!;Xpe*^4pP+R(DTBiCAIOJoG+gRnp`VD!Rn8MR!Y(ZFOqEnI2NwoT5tiqWk`u?-fIy@0=exWz3A8DRZ~a z8|fYp%0x1|$ib}yr<#$j;jEbe%E%@3!OD~O-`L1;P0Z*sGm|rdbAn2K<>*?JW^U5{ zAsH*^z-VIS&Jxy@#Y=(7Ki$2n-IzforLEg$L)$8kZTfI_y6NPp{0G8ntt<=6z1oU7 zTy+wzni|n0JzVy^-A26Lv48@h=P46jaiqGnmEo}j7bgwWZ8N9ZnZ-l37l_pAB7ExKaC;?^PL4Llb zM&#T`eL@F5)>@fc8&8Nj_}iAay*BX_a#`YhWs-7&LyE1%o*lyc$Tj3WXu3!3sEkIo zNpj=fRS%fv2V+wzx}wLHl3G=q)uPR8Fj1sD-i>QHb+ka;k){KRxZ&S|vdwO6zh-Vj zpQ)A74rts)t;`>cYK!a;&dT0;iB1Vva)*#%=HxvJ*%4zbS<{6b!YeF0E!}|uTJ$t0 zU#VgQhl}=|WC@I3D!F4t)|&&yYEQ9QiQ^?wPl-zEUG2K)0CL<{>O;3PO@!IBPf4c~ znw4$yPjeM*SQ|CNqU;ki%`sgT)V5@0AN7Fv9ZU1GOCmAVp(D3uTO8g#31Ki1y(R1z z`Rr&E!DHq58!kszQqXT~MB>{}LY~oF=4-{D49*C5s@o}U&mmk#HlT9hh}CJlYyJLufFmgK7yHpxFA8BNYF-ZgZg;3tmFQ@pD?pDfI# z9h{a|xo)Moo@f#=IrG~DCfAslx;ILYu{UZeo5c6XbUZnH*jiJ{ZlBF^$z%{S!H z3sIp=HtBK%!Ka)pWiekOg3XAC<&XRO}bG3fOG+4@$oQUaaTYE zG2J8mU>50eq9IF9Izb(MBztkLaF`v=PyXT2-1A^JC@AT>`=CE?`25Lde3LsbXEOweA5mkos+vnzPw=1o~$zlzb+v5#8hF%jEf!vSZA_|a&_ zYL;G=Qb~%$yWf!=TQMn>6o=n<*;jv8z$`N&d3QrVv5JxPaAGmTRv@?hy7AK|YaMjS zzqb-gW98hJ(08`F4aFYf|L%b=S;j|D?WGqztUYgckjhN(B4*S9sgT1BDi0GWzBi29>MpB$n$?%p z%9Xf;gqs`>EY~RFo0d~pntVrIC#n3DiHT`>xHo)v_U_|r`+L0a=sTWj@&hRxV`@T_ zc=d70!++$6QvPqhI`~(Ho4)&4J@J%g(EZME?>_Ios!YCm2mH^8-ktb9p2%B7;!0K@ zKJ@-`6H+Lr$h#vn)BBW(`YSp2qC;z=JQGUtj&)h;%I~1K1k5GUJuGvo^*XJLuY{90 zSIEmX=Rm1_H1wV2)z-;*{fx4NetYfiY*v8%4+U2(QOhEqW7E2`T4a7%H9`+k~- zRcVf{j)LI@59@>y>9LRbf1&SYxV|rfS1&!eqcpD7{asyR%;APo_|tC=X#HGwqmO1B>Cd@TindzX@N|*>x}U$<=0B` zv2j}4Rin)ai$_KlvmalcnIg}1aMDA0Y7c~U7WS2V)Eb5qVVAf$Wcc$4z22He9}vx~ ztV~)5M381CZ^^xKF~_wnRyLD!nEJ0xV=R1YUXKzgyr{zZ4_AMBrIM4Bf=BPi)lw*q z_lIl977mH$*!spxvay@h%R6cWmCyZ*&k3paT~Qch;0_MB%zXD z(z|vwbdx*J-ek1;&(j^*5$XX^y;&<$()B)#`a|Zpk&??L2`!su>nBZ;?LTa2y>67s zi8?HqO-f2a@?ITCxqe&8Dla-Mc(;lDXmb}`S@8wmpWt&Y)z&Safh4I zQ#-b(*!K{Y@dV9_gow!{j;Of#A=#V~jhhTx@DQsMX2UvuGqoxjzkLgW7vtv2d}4_= zi?GK?b&cFa6GXLP(2!jrQ+3A>J6I)0!_15xzlF?>KD)@b(@0uB+N9@&o{^ck z-bhimMBCo*e~+f0P1||J{jtk0A$!ecq_@P%9VrPN9CMVQDCl~IW<6!NQnqV@0j3Sr z_IV`kXJw`2b?VwY^i-18V{DM@w@og|e5JL5l;4%Dq8HDmjk4V)sM(aLo@WbtVyu5w zdYx;A$gt2d*DVqAolcFWyXtJSPQBrlT_0t$P3OQU%Kb&(pN_=F>Wi^q;zc)&yXu%a zYdSteV3I#(5|6Q;nJT#&?ApnV9REBd;WfwHC0xWDmf+Qu8u)mRGL-sWt2k2ea&Gxn zY(x`3x$m^&JwmjOEk__t->6Dka9NZTC%xoqX+I8}HCvu4kn)2^%G`^l*OK3ea<(cZ z-jJk;uL=_E4%XZ6-!Wa$G3rj-UbX$EI(D8L(aSm&v1!}3h8jw5Szv*jre)-j@s+(8 zuZh8?YM1L=s$gfQJa*ByB4{+&NPf+zQpfhq_@}mtxk{?IuF4Iq+jO?$d}>=KB*o8P zrsOF3ZH`6F1kMUsP1BaN#Q=_M7o|;axoZ)%l51hHV)OK!ZOa$dGuu-t6JLJ8AG3AQ zkxB*p`H7XNAP(#A15*b}W(k(j@oP@;rrx}{s)Bj*!eGM~c_pXQDtR5#4CPXL>a+$! z=fq9Q^>srjZ8o(1)h#Br8h2s|^CPTtsa_ys(L?iO0+giS@eL(&+Drtgr{BHy!dzSOuX>w-^G~4TlJnV8K3yg$lAYjc76SQ@$VO}pVt5J^XvsZcZR~c JmwxpD{|o3i^@sof literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.gz b/sql/core/src/test/resources/test-data/xml-resources/fias_house.large.xml.gz new file mode 100644 index 0000000000000000000000000000000000000000..3e0172271cc67d6cf612785ce383aa1147037be4 GIT binary patch literal 8568 zcmeHp_ghox*7nTNu@8<8DhP<9pwgvC6&Q8sO+qt32qmExLr5r+Fp8p5HG#xPQy}yr zgb=A2MS~#$q?b_CKq7Cj@GJgsTe3_o?3da4Y|pn#)t}dY@RcbgD+GA>nG9sA zzB`SD9RF9~-e-!1qN%M+9+Tu>dn)g#OREM%AioC+kPU>R{rZ}3pH4G7Z{tf z&6CGm(LHQKaH9+OT!nKWIcnbz1UivZP@10EDrr^rqE9g97U2 z`*kOQ;o7IgOK3_6I%0GFO`_rI)VfeQIOYR|x-xSYyw2grIv6(vUtIY#+gTs8GPSN3 zS&?XKAilhxH#@vK=9aa;RqwRn$8f>8#3lzXXxi8^=zbv)vkV)fZmyIp&84=Rqrpm> zgX;q!O{-Ds@LbO<`v;pB8*qOPQ7ky?;h({ULk*!u~E#dmTZm~mmCVTcLzl4-5ndPX<>gnyZ)3VzO z&9c0wf0j6x!_%~EAC))*y^I#=UNIegEXdv*&gc3VL`QQ}Q_aM=n6W!*a^g1a3aBGF zAooUFGycC6zK+LMVWp_A@B7jYSxISiR>PKrO*fX-L#XH_2IZhWr>`_R_jQqmj)L9U z2~7so2+Ng!4+~ZonJBt%NyzP|M19$*Rg<}-sK zk=qZfh7aCeAhg_cAFsfTQ~hrXU?(4mT87fZjs=VClp^;pMxA`F+Ppx&c@ms)wv4jl zhdbf6It%$4tcY=_()4^x&iI~8wcoC@wqs4OikeS&=BY~2gyFn+Q z|GdkSFMg(KwC9@0{Tu8j&~oPQZ`llxrB77s`=tkx^tKPT1Ng=AC{-TF0;kY z<>g6rKpU1aH&NaOd+c+pVA@NX5&A|oBsO&>tE|tN0tBg~)R20ea~DYn6vywIWK4}p z$Q88PO|~hEM&G3BxWp>5Ur4q)PN1g)zukzas=FEnn*$Z&mb#$qJnx4mi|cl#k>pMV z%+Avzh7-q@Pb4cMq5@4*E&`tAax4c` znat=(>VBhPSZ*0ki6;$K>+A&T@qB}0*E%xt5RXCU0oOZF$^~<%L?CX~?Toa_n90UI zv(~kiXU3D3E9k70XoC)CwKU#b$^8nPBXsp)p*_&;;%cC9;4M*6H0%Bu%A@1gFk%leWA$eQY=h+Lao1hGw)d;GcF|k$3Y|#KQyiw?B)ry}6Nt zuOo1ZiVW&_x=}0gSYD&GviY2#?&CSBY0r>X*IEUA4FsG4xAWqP+~n2#lfCZopt=^I zAauD=HQ6A@v%z4)wgJ&mAXas97@^WtVUR7;IoMOlj;`odeyn8roP@iaT^*Q&>>PO&fb>4J%D zfIbA+MDaic-7)9~r+3PrnC*Cz`UAo9h+Wi+!qgXJdr33Qig&J2+>^b4q7mycU_B%x zD7$gGv4`QX2DNPSp;gQG=O#ADAr*M_Om)BJTj&F}y_Zf^CQKF`lf;1lC<%3Zk#a8P zS{TMH@*?w0^`NP_big3ol3b@(2n)O!ec;+~;Jw@K(R29uRmo@HMR)bn2YdfwrPBKuBmIM>DW`2%(t2ITw{NlsL3e`+70IoH4rRb zUK_zWMp6b30Hygzll6uCffZ)=Okw1Z6e_p^vn`0z`{*e;OP=BOUFDmVwFBucdN9lMJT@K3Pz07PK=v2kI+^ZF|T$ux{&sf=SwQF4+e-b%;#yb3H%J9%Qqt{o37V%JHi4y1xMMi=E} z$b+(AY5g`KB+S&0vcupo;<{PmqbW^RrQeGG2(hKq@X^%DIhubLt0FX(NBn_+c#r$pwbu%!u9Tdy0F9V9G}#l{F~t6_T$X2aXKwh)J%a4% zDGE?PAhgtY36>xFtoD-@R~!XB=>q?Um28U?z=HzqBeJtNqhZDp_GE`Q<=WVY9+WKi z%Xp{o=zT@wF1^tu7HR3 zI&(yx5hge|MGx528q@4Rh~|8A*A}y@fTNLY#(dN1k6Ts%*%#|DU&IxzpbOYD82PxM zVvsa{DEqu_8)y|C$thZY7#0W+lQ_}APs#IQL!j*y8CN2KB?r&UXuEb8BI%8lPl0xG z)=?vL6{SR6l?9IHpP~K}q29{Ocf3A&VIfLX8eKeWlh*UC`DRan`tG!#={mB;(|;Rg2R1jIe^`#Q|aQ^%zi^)f5UUUPTd z8|fS7B%!Z7X;Q2R`jy_=lC%}$A@aJFWsBrr_e2!1`wB0&tpGMFS!6KwW zz**izX-aVpOi5)mOo8z>ul%nx;4B;Bg%1{tVsK~#30k|g#vopEy@z^Nm|lu$mcVS9GI?FhVlqCjT*HjV z(%OFue{s|E`x|8?);5p{W8Ja52?;VB>&jl)gZ@-?53!7cs(Q#cMS9)fU#L+>r1f}s zw;Nb6c+m5v(wY;{85{N!l}0{=w4IFdeMWs>@gQj@B(8Uh^I&wae%butSWw;xx0lcNIto*})o#7tT!}z)| zaU-_(L&d_bPV>oLnwy#Vr8q#^#NuOYI zbBLDvVif$zc?bJx~cI~N0Z#=8Xn97Aaby2Gg z+SttgA{=e8*~=xvUa9LL-zHwxz}UP(&NUg=GHS6;c#pRJcj; znvBZcbI_vx_$qT zkHeXmd>MJV(qa@~EOIRm$?EbaDe+rC!wjiau9tiEE!*Iq-pkW;#Y4c5WE;f_-Z$Bu znNpxN+r?Kk)#0|DxhAh&TY9q}8oDUVtq`X^TxkfTN9)qOOa{WSL@nvFapk^p*R(q(c^-xA3IWPa7kuM?5B+dfvOUx~NELtz^3H3ON_Po2=)e zd*OR8vtIL>#JZ(QkdoA!=aN)oxegi;rBqrnm3uVRJsB$}3r+{Qc5jV`eP>TUjw$@b4CU@q9$zV`+i6*gpzcFs#(!BU1iwwv| z(?cIiMfW+uJLw_Q280m7V5>IVS3rUbpZiVn27Kaej}J-qT}ykmb1w~TJSf`-J7k@t z@2Q2Wb9Kdvr{&f507iS+*UUx~bgs<>6k+ycLRYwP^B)3 zH$z3Vr-|cNfSVIztyN4+o_|}x)M+mZS5KH4Ebk41Z9ObBL-{$!-zkxp_i?+fv9CC# z%|TG`mUiwuCP6sBNhCSJaAU@O@4H#pvxeMv^NVninPtZ?q>9iVNh$9loR53SUyiDM zqq>q?@?HJIC>vKwt?iB*a>n*y>{iyrWd6wKw*#jLPdkV`S82p$^;K&t##8&Cr3>2Sjp&;- z<|9&V5HfD($&%R|q}F62Du_Kfeqw&nG8|Ps7ArCnbx`*?|3vi~AoqtZmcDO5W@QC} z1em$=iMaG6sj0JLbl3rR#8|gak>>^zYDZVt?X@8aT(8(if>*MlUd)zPl?RU>mmA=~ z9+jB14d-P$w3~U!Bs9-(dPlMWn69_e%5EUk%%!YKC(5L9$3<}a@oWg@1+NBAqMaoS z2i)D?^@H5f2hVmNlr!HwZcN*?Fv`raK+PeNWpXqH@0BIj`XDu**53G9eM{9i?OLiv zI(*6tf65Bz|L{UFLPNm=9KP^fsl5UWv;;Z_ZLKSB2Aj!@E??5egZz_IFiFle+MCKoc1=qZWd(nC1_o31&$X)%K{-}O*o_O!j#wiXr>x7~%qG+I}rwmN1 zYzVUy&WXz0zdFCVE(h+Gq{%_%*x&s)l<9ZkZfj-xTv?Z7i)Od2z3e3M6dZkAI!l84Hu|c2uOt5tFd z?;P#(G3aL~Z+6XCn2AJuEoO@?RKL1NcN;EQ>vrWjxdzri~_aK)9G5sK*f&O zZ?1xJ*v+R`-{u90kXT8vb}HH%)mO-9*O5R*4$Zpt8k}QLE|Dq*uz=qO&qoXZxfCpG zx$PAS7{_X6NwS?A^y?7e_s5d>NgXR<&2Y;EI>}7mlt0xyjwwx0vuhP~!8XYs4Jy}^ zwN`vfek7kc0|7{|-0Q5_D0E`qXJ)`awq8cmi}y#Nlp6c9dyN)gCQcUl=NfGiYLp#- zJ8un1t>=e@KdoWKxk>Z)k@G6{lNstAlmKW}`UvbNQ%pmPCP$&*wxfK;{0HksX$dDz zG0UyvoEv&-*wF2lF}JB&L~LFCcX@OX)dE}hLBDo%*5i2}TFLo+wLHgZAH&Go9vg9I zV*d+-9{_tFANVg|b{m>#wZDa&cnkR*<-7L$tva)%YuX>a*J{`Q0FKcv6YQlwp!T@2 zubQt~OE&(;KfL{;#+T2vS>$c|-@)?u`M)s!b0&nX2nh|sjm0OOf1vm~*|-=&mo_zDdhl;V+qBE++iFAI<)sJ0Ue+8*h2h35@yGtLXn3GmqQ;+wHdN zUqSeuCG|U%-%j;kq(q85Va0J5drE&omyB`y*Z)w2|11-Fdh5XO->Lj|nvVMtB_=0QgAnU+d#iWgYms~Ze?@+)%>N<@JYd3~ zdXFTv>$f1kEGt|om$0=y2akWDdYm4j%2rr}k-%RkNT-nzza6bt&7Vv6gdRxUPSk(G z8~&}BmA8cZwwLC`-T&4w-tL=ilpZqrO96E%M{fR!&h`M2B)&)~o!bc!i958_JIq|e*}e|s+XQTkKwF_B`w)$q%*A#`M`KeF_Fzfe7b9lmKh3?xbF-~2iA%G?_xxKwgtiOr8ib&&v3?5?VM+h0bLam|NyegoNj&Eja`&I;Y!4DM_=}VRvrMo$ zV*f10--qs>#rV%+{Lh8<>zjWT + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.bz2 b/sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.bz2 new file mode 100644 index 0000000000000000000000000000000000000000..7e55107549a3b95974e0a66153a150bd60f8d76d GIT binary patch literal 4571 zcmV<15hU(HT4*^jL0KkKSy5Uf%K#yY--~#VPy_$>zwh55Kc}BSKnTD9007_?KYhS_ zmC$+$Xc_{dbQnsY018kEPzq7p2eya>HLQK-&;U|NP8ynNpwWm*RKOvl00hv$0GbU< z0RT#sG8!}h00Te(0xEe=(jJmDXa_r z6_J7|n^n-E5e*!OgBS;50E5c3U7Uv5RmI8Gf)x;EWl|#}DqY#Y>cxG};J24sG zNGL7bQ9>eItr!eSrFv#17O)N$Q##O+hd_e)B$Ehga!Tq@L=Ig`D9Yw2JR1>JJfq7MLYcElAsNtMPhX~# zgh6+iy3Ff!)pf;Qb8j7F%xWX5D|gl5S(kmcciUX`+p)za)0lEGMInr)L~n}D+9@DM z{XNzmYHrt{uqtfF2JGh2(GZDKXmp8WB_)@ELL#CF0VE!(R0tC;N(7LxT7eNuMJXc6 zRG~`tU_c?X0(a+yVO?B6$i{m>#P~IVtFmnd2xk`9(D)I8*bG2mHHD0pw90Ta<(=(| zqUDS!Vp-LIG#RaeWz-&pW{i}c!OsHmK}3_;8ukZaq6k({ELn3?Qu(AInc%~a7DBNX zG>%2Ngxbz8gb;vSDBELtL(rKIryG}R%8e5R<5tKVN+}cxero&-oXWdS2yc}o%BZY9 zg0V2&$+)EOSDTIpj9@9B9CDzn(76h{!@#_g^JKhI#?MZO6l+>_2qxzdsTh#_D$Ft-=!>S(9u%5cxHU5w z;Z_)Hfoh(X`ECW;Mvp+yY8V?{@*`LnG~Ey_;&AU-W60*mHWz|)$e4Cv#J2CqT<)*1 zA1;n=7D+JNs2m&P#`rmyOcpBQ=+Vq($an%T5y!pjZYu^(&v??3=t5O8g7j}m2?71= z001=^0FV|sZ1}gRF>4C}3)F>johnR#MPwidgSJwdV)CShbkwZF_AzT%KtXY0lChJ= zTs;r9vzy15=z3&bIR=mb@nJ~pW;QSY4E6L?5;uk*(iQ=h&Xx*`0Xk1zC?Rmi z7!Oeokirm(P;z5DV;8V14%Mudg#y;*Epcnv(a8KGAf&|-f*q?EumFfz0Rdn+uQ1Es z7XgpUVVV%zdn$_p4b)dKV>^b8FM%MOPBh8CEr65CACA*`^+$xnabKZvz z5JHBqrld8xPGo7K7LKHJdLXe|F{4WAa)tEz+1-fzLwUl58PV$Nx-NA&F_>7q&j?UK z)7lyKK^$Punfhhe;ESc?FjQzoU{(#|s=u!C(Q6PZesOh$i$Ez>VTQ0^fLLIx!U*hE z5J9I=8wNEPUEQrY%=BHD8Vv#gwE$X}8J9MfRSX`1_%jzE7_4C&!dN?#1hCxt`{Uz_ zTjuzzHbAKX04!2x0)`_8r-V8;H*=iYdO`?cisMC|eZ}`oFty(ru~ z<667W$Z!y0O!|42^EeaM>q$VCcz{C?GPYz|)|N8c+0kqTB3cZfmkw{DsM$FxBPl894B|o- z=D5`x-ZVsc6_(T`=*(pTQSx~XkKu2*z+2_@+w7X;9ec7=F<^%AbPwhF&w{P6ADN}D zFXYz2>`_fun~0Q9@M%6|TJ=9CuNr$V3;ewif-9 zu!)PdhDT~En$1%hbG(WdEVl9wz6+sQ_+5=7ikP^i8h}G(G7iFlBb$6-a8T<;CREs7 zch*x>dzAtff3dgMUIuRZEjKbwdg|gJx>q3ELn~uEhQ;eO96^;a%AlU1W-mkK2dGr% zaleHaB<{?G(U3EymefOraQP2Vjmv5V*`z@BmMn2=BC7UFF&&MK)tQ)K2Vjgq%)UKw z*vE*ZrxA5b8{I8}0U4EMz{$_#!BGQ}poXrJLUI&X0!s^z1{!@z#>8*FG1dsgOLy^AYlRL2jaFzP602KI}CDI^Sff>1U+ZB7&t(8Izaat zNt==hNIc;Z&xs{LO z8B32M!ViL4-79dd32v&|vjF1pOIKbr= zQ3P*kpT}zSClsxSjR26g|U|#*j1c7Z{htq_7TF8lUsQp{O`z&f3OwtopOhWVL}h1xvMFvr< z7<$HiY>MnAEO@|PkUffx{50t+Ga_c>=VN>l)MjE_kDeN8`xiE{_n&-}(me872J`gf zs|g_ro2?z}A$V8KsV+O#D3oHpCto|@KhJkvt`peYg1oGeLkdJ$!N;|z+#Zor?HH2~ zmG^ocpShM;VFoBB-jBD60LU7Y+m>;?WiDAM5(^Q7)kdtsviLI0K67!F<9d4d<}o1t z{H=}nAmRE@5YcP!j9duZ zA>$UNeB&h_rN$khqQD3!92biIQiPS(^t9$oVOcX;bE?Uq7HrOZ>5PtT6L~$@hnH4S zQjdNxxN+-X@wqAkleDbtX83&-H&80FVdSUpubQbM$CBIl`XMvM1VNCKp=hfVT$F2Q zn3i+a3yo~U>FbLm+rB>Cbd61K4ezh%?eZcO$|4@+PDAP(Wdwl}j{=Xd-_q}(vuMmNw}Dk<6{8I$ z-4!h57dB*7F79Z=FhYvWo2}7X8=#k|p@k;2IxkyBWnJNP&=zJGYN*jE3KFQG!wM|b z*ljRKmz1KoFx7@$W!2lGM5&0z3kZlK5ClLzwy1o-hh&@l_p+iLsDvUEf+5U^P$C?e z0L;+{h;u5AqU;#T0+D4wB~?{Yr41ELRZ~!^YKEdRQl~;8(1=+OxnPkbWRQeO2_$s# z_`q=mu-$100yL`;4WSUku?Zv~L1bD)FhvSPDG57>5_Q}w7%B@9j7A`Wut8KsV;JO; z6TFZ-dHBTcSAx)gBj!dJ#Mbv`_$iESGb`@#rAm> z&gi{2QMeP)o(wrjG?&q2IgnU81yVJR=yKWac=h4$Ty%rKFk;c$#Gju?cySWg)Ig0O zaN>?TH%@y2NZ61=r3OsN*E=C*+2jvvTuDu0M&jvt)kRZgtI5+dZf=9;Tei$Lgl%1k zxM4ozaw~2mgtA-N$yIPe#LGx&axF(wSRF4q+4EAj^sT{LkFJ|2#%%!Jf?#=q1RE5l zOYkd^Yk>s}^@p#;1&PrEGbRW3*SXzm?4vqqA5-c`S}>Uy*3z^UBGIpb-Jj)}Sd|_g zwmc658OHVUXB}=7CahSqaj;%^9AA4aaAQg!@H>dsC_9PZk+JSK#C_1L>_z$TM_pid z-SP{uH1?FWjox3d>(vC@?sDGll!AOnA=a|2?uDApy{gE4e5NA$^Qj<_13(*w){yUs zoG|*orMaUu4ZFE0&UKm>EYL9KaD{|+VB1K3oaZP=ez@=Ab}mvjZL%tetF?&-eDRmyI+~ZZJ$l5Z zAd}8s8EMWlo5goH%Yo6Q%dQZJZ;4FCMH}3VeqTIk^MGA?fDko!~%=p4d? zfQ9UqT$AM1?}<3m7S17Q4UG)DU2Ucnsv&F?(<^{Sj%sWw5I{4_a8#xa3N#RGj(aCN zHKrj}mX&1&_l=Em=u^=}RAHlI>o(Q@y5iMBS2CfRC{!%P=lz_{V))PgF64@Ep&+8P FNtQYRMX~?@ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.gz b/sql/core/src/test/resources/test-data/xml-resources/fias_house.xml.gz new file mode 100644 index 0000000000000000000000000000000000000000..d61afdba3931f0b9f1ccbf3fe90a3cb37e3c3083 GIT binary patch literal 5069 zcmV;;6Ef@{iwFpOz(HLA17>Mqb6;q0b#rAdcx`L|tz6rVB*$@m*HfWyEuAZ&s?24qOYNooU>qMM5 zCn9S9{@1_#{jY!f^0)uddCHv&Jre*h&pg z%a8xl`@c@9K z{P^yxckh3E{e;i<&5!RcM_#}8)w}CA*YEi4@4ox-cYk>O6s*V3>CyWSM{oc6t9Ng4 zue`TWJbq!P=idGCbdGH#k2m`MEnfH!U%!3x&Fd#}9DO{!-4E~HeDw~eT(nBN1uikw7FYo^K`)~Af z{CXcIH}F3W&ETckkdth3x8;1-rH#gR1Jj`k_FxJvMgclTLM6a7yeV9&@tj5$Yx8t% z4HJcz!el+?D@@A~Zp{zZ?+3Y_dH>XVkjVWB=?H1lFzjP9->toxJPji zx^2;>Vsb0NP)$f8Y3Ty%EQ#_a4nppT_1R1v>YZbn8Ab4Rf z_;7zi%8=)!FvkY>0$7zp^`Y@AyMg7AJMgwa}ygqP`r`)UhVy4Y?e=fNVBInfbD z5OTSQfQXVHqT;kh>YPg?xZ0)gHWGki&mj$yY$=Au2u5r##*LX~cA;38cOxOwz-6(S zhF{Gfud1nuhT)Q1&C~Xo=Oz+f8V634Aukv&M`w&P_!$@@+x-WOGok2^a`Yy>eIgfc zQ4Pj|e;@Mp74?s1`XP|q7}9#rhHXc&w7!^{gI_@MEy?7z0SN)zC@|QeYLRS3Oo>$t zw;Fnq?zryLHef(T+G=9Jk5jM7)Jnnn())2hiX6th1 zQh^g_`*a)fwE`!lZBO|BhuO*~5sJA5!f@^-*vWbI{=diO3d@RH)(WoYC&WZGtikqnow_mL@a}>0SMJ9570E0t%)UE&eX! zb-F*3cJPqLajUk62t&YGkS#Dpy(g7%!w_Nscd1w@V>b?JE}i>mrYAQ+Bn`2HYd{Zy zYxk}kpu&GNe#X&}1mWNHKU3O2{@_Z5Weh&f9zYaeOHhfy^T_Cf*!%2^n9ST=4C) zmo48Px?m4qi#k}TbORt!SvXHsW@KczmS^(QmcGgJ=lI?oQfE%wN}ZjKHtNh-f1}Qv zy*e)NwuKr?a1bA2067ft9vIH+VV(!#6z=LcQa2f8nqMZ<>5(|AzE@_{&Zg#;@t-QN zPLDKw} zTWleYC)5s7`SQcOg4j8`k1-@RD&m-?%%mJVoXKV4-r+YVEt9^;5sZr+ska6~Mm+$= z6AU{!T`=yU*15sBf`w83!8a4Kof!67D1rQkD=xvo&hP+Z$nV~xH*?gi<}n&^Tns}E zN*ZvF)or-^r5L9uD~y+;b*iVR%6gsZ%b9Z=B=*5ChV(-mB=|!u=+BmPO^iE7{z)Z$ z$B};`h1UW(Ha&DV!SYfX9vXVYGpq$NCG;(}_Pmm8K&g)Q}a#18QAta9nn9 zM)_XD`do!R=C8e)qz~+TI@&O}cef0FI&)!g2gM;E^;M$j&+fgK zy|3c;nEFq|v{@Pnk!R8fsPD)#*f4fyh?-*{vBC7v1-r5f>f8HTOq;4CrW1H&JTOc0*ORI(mvyT!_^m_%2If}a_WABjT?l$xIbmPNA*_BR!#TyFs}@6fdy0NkPW8-4fD_jmU(^dGTYp3erBg+8H_P?0y_rh^{UW z-Kk}5>;Q8S6XOZAY0Hcg+^ zT~4!pC)M2^#@4uXs)w+TajMc>4mto){~CNWxJ#gz0lzaR^lgtvFrM6ICEV&f)IujP zMfw>~1xeh8?_6DUkG#|9aoCC2iDe`Ra!ICG;Ef_4@p=Hc`c1KZ zv3JULCBSfWZIPfD!8@gmck(Hni}wcKz&7Mte4~=^jSjoPcVX^(e3$drS{)8`G(nqF z3aC=~l12gP=6>#E#fE?RMkXJ*U>7AAoR&q$Bwjh%R^%^Sl(fv111CGSosJhvs>Exm zU5+;Pht+9r?9b`U#r}Yfy|6YsuJ#9X>M)CZGr7tXWgLPj-IWnk34F|h}p$jS8+!EfPUub&v zQqr~)cIbj#NmgQDmS$X27e6`zix61R^RN&W$vfV5I-je>#{wj*HTQCK0YBX5u?}YL z^O)2l%Ohv>D34Fo+3z+$+-YuoooU!Zn+2V1Lni4k*)_d=*;MP!t*Ydy=eea@l%eV~ zLM=4WZUl;ICGA(?qbG~dPeXQrLP(_&D(HL&gufLFZ8P~XxzI!4KHQ<@5vqclC}4fbcg}@o;i|n5Yw?!OfHYl z+o#^lB9^0#P@~(;vE|IAR!1Mf?$O$Gt<{mE3N`1S+Z^M#%*{XJ=$Rfgy&mJL3wf~l zrrFa>TGw)V{DA8W#oiR|BNyy9y>uuxpw*?ps0H$J-c|wipB(Avbd)WoGTgko*Z;+Tniv}b^!W$!NOH5FNBW@N&bkFl2 z-+%juHy^da+Xm1U>}y-o*_DJ|jxK4LeOQ(S;x#RyP*G~qeLG8#s`>UE@5YGf6UK-e zx|Ii0L^x6yylizD1S+^tG~z4#6#3HA;1J3oADxGVf9b{Ux zgX&W^pZ0at$f)hO8AU@A!2qU4m?W|=CCP$Y!fjH&aQtL9ZE3q|OGn+sJ?+?E$G}n8 zTT&EmFm~c&xMPlwl-Fg!N8Q=p;*Sjn+R~8{uyLxGo38#SfA(H*{cNs8PN$a4$1&E9 za2!6vo9o1oii3QTbNeELx=Y{oJJMvD4#L+tTW|q_vJhA#Di!B4P*wLgJ)!1H;)GwZ z>X3uEnOuvi%8FZ<@{I8_x24(fdTx9=`mohQ=k<=&h|%t~I|4qUg8*=k|e>`*NTFm*Cz@w1uoG;Ks=aBCdu zepf0=8UeCbf6^Js0d~yJYC`1*60tOSvp(Q_ma%D{DJ2-%%nMe9Cj*RaMunN0F2Y&u5tEj> zT&N?X1m@B~-nD(KpnJi=;MdYX1ca9c2Q)Q21o;$44{3bXIe<@o^%XwUs_Q1SRxNe_U6XjOm4NyC+4rRcHsQ>t? z%vnKFt>r`d$cK_PZA;jU5O!NpnIoT)~e zDn1SB@Zt94!RH*vt)FwC>sEB8dtS}-XqC>hhQi}~JuPymKe~aKlzI=W+dR;~5n{w9 z(v`QWV|&tv#wM6y)KVH3U7A1Acj^w=DK?p*tG&nZg(8 zTN@{vx6=2}1^ZoP*GRbrg01Q>*QsXi@UxZ(#t6k>vw8L6r<5QK{bup%=D7!)2{+qx zr!&{(EPav*!AtAqEPaxRo*OQ}Az8Xx%=x6J)pj!c!Em9uDZxoG4YyrcJe|foCLttY zYC;`79T_gbMwL`7%x0CLT?#Q7T4X`BsafHz?i$PiY9%dopSTp0QHD+6u2WRlnXup1 zNIB#@JKXrt=U?i;SE(bMs74PZQLn1uj>I5 z1~W^P^B!zV!Ij`|Sp2{RyZg|-xs-HpkWm9D#*7PDpU)BrO`Jr#(bn$86C?_L<#Ly! zb*@81-!z(+GneI^;62^+BSYZRKJ;S-cOg0VqJE@TwYFWv{M6~ebTXjWtQjFr9m~Tp z)p-F42&K5~PsJvcbqXm_hXAlUrs_~RwThRLnzYTdhbl-<3Uc1)!*DvtgdW!=ONjF+ zNP4)b?K<>}yUg%DtIS=FHmFj#84oXKE~p}lMciiXE)bKP*4!Th>u_V@oeV#<9*;Mg z?>yCm*`O8cdZ1|rQ_*J}tdY}}R&1zPVb-1xF?fwV`k;;Fs(&qj9Y81Y`LBKGu+3a) z69ajNa!n zTKdc@Z|4q&F4%2Rju1*C5Q-hTqwByFWehW2AZaK9Y&WA{krxZKO&fYS+6Xmqw?gf7 z=DJ;rJolS%1PjPp+nCQx2?G?4-0jpyZCv>qx4q0L?w>l3=LP)rQqQ9e>Ossf*_^Vd jzzU}3SWSmN^U<;jxmzpt<*(0wG0gu1=1GygPDTI#`mq+k literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/xml-resources/gps-empty-field.xml b/sql/core/src/test/resources/test-data/xml-resources/gps-empty-field.xml new file mode 100644 index 0000000000000..31bb29ff349d8 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/gps-empty-field.xml @@ -0,0 +1,20 @@ + + + + 0.0 + + + + + + + + 0.0 + + + +
119 +
+
+
+
diff --git a/sql/core/src/test/resources/test-data/xml-resources/include-example/first.xsd b/sql/core/src/test/resources/test-data/xml-resources/include-example/first.xsd new file mode 100644 index 0000000000000..6e6b295352b4d --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/include-example/first.xsd @@ -0,0 +1,5 @@ + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/include-example/second.xsd b/sql/core/src/test/resources/test-data/xml-resources/include-example/second.xsd new file mode 100644 index 0000000000000..c22972f4c205b --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/include-example/second.xsd @@ -0,0 +1,15 @@ + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/long.xsd b/sql/core/src/test/resources/test-data/xml-resources/long.xsd new file mode 100644 index 0000000000000..b582852964e62 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/long.xsd @@ -0,0 +1,10 @@ + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/manual_schema_corrupt_record.xml b/sql/core/src/test/resources/test-data/xml-resources/manual_schema_corrupt_record.xml new file mode 100644 index 0000000000000..79b83ce6dd2b3 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/manual_schema_corrupt_record.xml @@ -0,0 +1,30 @@ + +1234 + Mark + Mark + Mark + DOLLAR + RT + USD + 1 + 3000 + + + 20210207 + NO + 20210207 + 14503 + USD + USD + LEGACY + IBAN + sm342 + + NO + M + 46_STREET1 + 0811241751 + 46_STREET1 + SA0010001 + 1 + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/map-attribute.xml b/sql/core/src/test/resources/test-data/xml-resources/map-attribute.xml new file mode 100644 index 0000000000000..b80d1fd06d4fd --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/map-attribute.xml @@ -0,0 +1,7 @@ + + + + 0 + 0 + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/mixed_children.xml b/sql/core/src/test/resources/test-data/xml-resources/mixed_children.xml new file mode 100644 index 0000000000000..9a3efe71919c0 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/mixed_children.xml @@ -0,0 +1,5 @@ + + + issue lorem text ignored + ipsum + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/mixed_children_2.xml b/sql/core/src/test/resources/test-data/xml-resources/mixed_children_2.xml new file mode 100644 index 0000000000000..6d602f86d8fe4 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/mixed_children_2.xml @@ -0,0 +1,5 @@ + + + 3.0 lorem text ignored 2 text ignored + ipsum + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/mixed_children_as_string.xml b/sql/core/src/test/resources/test-data/xml-resources/mixed_children_as_string.xml new file mode 100644 index 0000000000000..ee2e4a7b35f1e --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/mixed_children_as_string.xml @@ -0,0 +1,9 @@ + + + + Lorem ipsum dolor sit amet. Ut voluptas distinctio et impedit deserunt aut quam fugit et quaerat odit et nesciunt earum non dolores culpa et sunt nobis. Aut accusamus iste sed odio debitis et quasi amet rem quam sequi et voluptatem placeat aut voluptates iste? Vel nisi rerum sit eligendi excepturi et galisum animi et ipsa nihil vel consequatur velit eos velit nesciunt. + Quo voluptatibus sint ab officiis aperiam non obcaecati rerum eos veniam iste eum ipsam modi. Non voluptatem illum qui molestiae magni qui maxime commodi et accusantium similique qui necessitatibus minus? + At quod rerum et porro nisi ut tempore error et enim optio cum Quis voluptatibus qui dolores sapiente cum cupiditate quia. Ut incidunt neque aut provident quaerat qui quia illum. Ab esse commodi ad earum molestias non internos atque non consequatur inventore 33 galisum nobis hic distinctio impedit! Est dicta iusto est numquam incidunt cum autem temporibus. + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-attributes-and-name-of-parent.xml b/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-attributes-and-name-of-parent.xml new file mode 100644 index 0000000000000..b364b499d5489 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-attributes-and-name-of-parent.xml @@ -0,0 +1,5 @@ + + + Child 1.1Child 1.2 + Child 2.1Child 2.2 + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-name-of-parent.xml b/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-name-of-parent.xml new file mode 100644 index 0000000000000..11824c3d250b9 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/nested-element-with-name-of-parent.xml @@ -0,0 +1,5 @@ + + + Child 1.1Child 1.2 + Child 2.1Child 2.2 + diff --git a/sql/core/src/test/resources/test-data/xml-resources/null-empty-string.xml b/sql/core/src/test/resources/test-data/xml-resources/null-empty-string.xml new file mode 100644 index 0000000000000..a51ad7e6ab39d --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/null-empty-string.xml @@ -0,0 +1,7 @@ + + + grape + + 5 + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct-2.xml b/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct-2.xml new file mode 100644 index 0000000000000..5e7880a6a3191 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct-2.xml @@ -0,0 +1,49 @@ + + + + + + + + + + + + + + + + + E + + + + + + + + E + + + + + + + + + E + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct.xml b/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct.xml new file mode 100644 index 0000000000000..dc90ea26660e7 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/null-nested-struct.xml @@ -0,0 +1,20 @@ + + + + + + + 1 + + + + + + + + + + + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/null-numbers-2.xml b/sql/core/src/test/resources/test-data/xml-resources/null-numbers-2.xml new file mode 100644 index 0000000000000..ea620c627940a --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/null-numbers-2.xml @@ -0,0 +1,6 @@ + + +
+ + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/null-numbers.xml b/sql/core/src/test/resources/test-data/xml-resources/null-numbers.xml new file mode 100644 index 0000000000000..926cfb33c37fb --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/null-numbers.xml @@ -0,0 +1,15 @@ + + + + alice + 35 + + + bob + + + + coc + 24 + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/processing.xml b/sql/core/src/test/resources/test-data/xml-resources/processing.xml new file mode 100644 index 0000000000000..80ba2013acad1 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/processing.xml @@ -0,0 +1,6 @@ + + + + lorem ipsum + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/ref-attribute.xsd b/sql/core/src/test/resources/test-data/xml-resources/ref-attribute.xsd new file mode 100644 index 0000000000000..d5793a2cfb186 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/ref-attribute.xsd @@ -0,0 +1,19 @@ + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/self-closing-tag.xml b/sql/core/src/test/resources/test-data/xml-resources/self-closing-tag.xml new file mode 100644 index 0000000000000..c3057b220bcef --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/self-closing-tag.xml @@ -0,0 +1,6 @@ + + + 1 + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/simple-nested-objects.xml b/sql/core/src/test/resources/test-data/xml-resources/simple-nested-objects.xml new file mode 100644 index 0000000000000..6e62f64e704af --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/simple-nested-objects.xml @@ -0,0 +1,14 @@ + + + + 111 + 222 + + + + + 333 + 444 + + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/struct_with_optional_child.xml b/sql/core/src/test/resources/test-data/xml-resources/struct_with_optional_child.xml new file mode 100644 index 0000000000000..43d84345b144f --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/struct_with_optional_child.xml @@ -0,0 +1,8 @@ + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/textColumn.xml b/sql/core/src/test/resources/test-data/xml-resources/textColumn.xml new file mode 100644 index 0000000000000..e60cf189d0b27 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/textColumn.xml @@ -0,0 +1,18 @@ + + + + 00010 + value1 + 0.00100 + + + 00023 + value2 + 0.00200 + + + 00025 + value3 + 0.00300 + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/time.xml b/sql/core/src/test/resources/test-data/xml-resources/time.xml new file mode 100644 index 0000000000000..0374d3e485b00 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/time.xml @@ -0,0 +1,7 @@ + + John Smith + + 12-03-2011 10:15:30 PST + 2011/12/03 06:15:30 + 2011/12/03 16:15:30 +1000 + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/topics-namespaces.xml b/sql/core/src/test/resources/test-data/xml-resources/topics-namespaces.xml new file mode 100644 index 0000000000000..5a36546a1da3d --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/topics-namespaces.xml @@ -0,0 +1,7 @@ + + + + + 1 + + diff --git a/sql/core/src/test/resources/test-data/xml-resources/twoelements.xsd b/sql/core/src/test/resources/test-data/xml-resources/twoelements.xsd new file mode 100644 index 0000000000000..593b25e343237 --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/twoelements.xsd @@ -0,0 +1,5 @@ + + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/unclosed_tag.xml b/sql/core/src/test/resources/test-data/xml-resources/unclosed_tag.xml new file mode 100644 index 0000000000000..04649a3dcf9ea --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/unclosed_tag.xml @@ -0,0 +1,4 @@ + + + + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/whitespace_error.xml b/sql/core/src/test/resources/test-data/xml-resources/whitespace_error.xml new file mode 100644 index 0000000000000..cd0ffcf08adcc --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/whitespace_error.xml @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/xml-resources/xsany.xsd b/sql/core/src/test/resources/test-data/xml-resources/xsany.xsd new file mode 100644 index 0000000000000..a131ed3056bdd --- /dev/null +++ b/sql/core/src/test/resources/test-data/xml-resources/xsany.xsd @@ -0,0 +1,37 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestUtils.scala new file mode 100644 index 0000000000000..1cc111517f209 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestUtils.scala @@ -0,0 +1,37 @@ +/* + * 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.execution.datasources.xml + +import org.apache.spark.sql.types.{ArrayType, DataType, StringType, StructField, StructType} + +private[xml] object TestUtils { + + def buildSchema(fields: StructField*): StructType = StructType(fields) + + def field(name: String, dataType: DataType = StringType, nullable: Boolean = true): StructField = + StructField(name, dataType, nullable) + + def struct(fields: StructField*): StructType = buildSchema(fields: _*) + + def struct(name: String, fields: StructField*): StructField = field(name, struct(fields: _*)) + + def structArray(name: String, fields: StructField*): StructField = + field(name, ArrayType(struct(fields: _*))) + + def array(name: String, dataType: DataType): StructField = field(name, ArrayType(dataType)) + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala new file mode 100644 index 0000000000000..c08f2d6c329bb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlPartitioningSuite.scala @@ -0,0 +1,75 @@ +/* + * 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.execution.datasources.xml + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.matchers.should.Matchers + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession + +/** + * Tests various cases of partition size, compression. + */ +final class XmlPartitioningSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll { + + private def doPartitionTest(suffix: String, blockSize: Long, large: Boolean): Unit = { + val spark = SparkSession.builder() + .master("local[2]") + .appName("XmlPartitioningSuite") + .config("spark.hadoop.fs.local.block.size", blockSize) + .getOrCreate() + try { + val fileName = s"test-data/xml-resources/fias_house${if (large) ".large" else ""}.xml$suffix" + val xmlFile = getClass.getClassLoader.getResource(fileName).getFile + val results = spark.read.option("rowTag", "House").option("mode", "FAILFAST").xml(xmlFile) + // Test file has 37 records; large file is 20x the records + assert(results.count() === (if (large) 740 else 37)) + } finally { + spark.stop() + } + } + + test("Uncompressed small file with specially chosen block size") { + doPartitionTest("", 8342, false) + } + + test("Uncompressed small file with small block size") { + doPartitionTest("", 500, false) + } + + test("bzip2 small file with small block size") { + doPartitionTest(".bz2", 500, false) + } + + test("bzip2 large file with small block size") { + // Note, the large bzip2 test file was compressed such that there are several blocks + // in the compressed input (e.g. bzip2 -1 on a file with much more than 100k data) + doPartitionTest(".bz2", 500, true) + } + + test("gzip small file") { + // Block size won't matter + doPartitionTest(".gz", 500, false) + } + + test("gzip large file") { + // Block size won't matter + doPartitionTest(".gz", 500, true) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala new file mode 100644 index 0000000000000..6fbc8b98dfe1f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala @@ -0,0 +1,1549 @@ +/* + * 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.execution.datasources.xml + +import java.nio.charset.{StandardCharsets, UnsupportedCharsetException} +import java.nio.file.{Files, Path, Paths} +import java.sql.{Date, Timestamp} +import java.util.TimeZone + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.io.Source + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.io.compress.GzipCodec +import org.apache.hadoop.mapreduce.lib.input.InvalidInputException + +import org.apache.spark.SparkException +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.datasources.xml.TestUtils._ +import org.apache.spark.sql.execution.datasources.xml.XmlOptions._ +import org.apache.spark.sql.execution.datasources.xml.functions._ +import org.apache.spark.sql.functions.{column, explode} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +final class XmlSuite extends SharedSparkSession { + + private val resDir = "test-data/xml-resources/" + + private var tempDir: Path = _ + + protected override def sparkConf = super.sparkConf + .set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "UTC") + + override protected def beforeAll(): Unit = { + super.beforeAll() + tempDir = Files.createTempDirectory("XmlSuite") + tempDir.toFile.deleteOnExit() + } + + private def getEmptyTempDir(): Path = { + Files.createTempDirectory(tempDir, "test") + } + + // Tests + + test("DSL test") { + val results = spark.read.format("xml") + .load(getTestResourcePath(resDir + "cars.xml")) + .select("year") + .collect() + + assert(results.length === 3) + } + + test("DSL test with xml having unbalanced datatypes") { + val results = spark.read + .option("treatEmptyValuesAsNulls", "true") + .xml(getTestResourcePath(resDir + "gps-empty-field.xml")) + + assert(results.collect().length === 2) + } + + test("DSL test with mixed elements (attributes, no child)") { + val results = spark.read + .xml(getTestResourcePath(resDir + "cars-mixed-attr-no-child.xml")) + .select("date") + .collect() + + val attrValOne = results(0).getStruct(0).getString(1) + val attrValTwo = results(1).getStruct(0).getString(1) + assert(attrValOne == "string") + assert(attrValTwo == "struct") + assert(results.length === 3) + } + + test("DSL test for inconsistent element attributes as fields") { + val results = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-attributes-in-no-child.xml")) + .select("price") + + // This should not throw an exception `java.lang.ArrayIndexOutOfBoundsException` + // as non-existing values are represented as `null`s. + assert(results.collect()(0).getStruct(0).get(1) === null) + } + + test("DSL test with mixed elements (struct, string)") { + val results = spark.read + .option("rowTag", "person") + .xml(getTestResourcePath(resDir + "ages-mixed-types.xml")) + .collect() + assert(results.length === 3) + } + + test("DSL test with elements in array having attributes") { + val results = spark.read + .option("rowTag", "person") + .xml(getTestResourcePath(resDir + "ages.xml")) + .collect() + val attrValOne = results(0).getStruct(0).getAs[Date](1) + val attrValTwo = results(1).getStruct(0).getAs[Date](1) + assert(attrValOne.toString === "1990-02-24") + assert(attrValTwo.toString === "1985-01-01") + assert(results.length === 3) + } + + test("DSL test for iso-8859-1 encoded file") { + val dataFrame = new XmlReader(Map("charset" -> StandardCharsets.ISO_8859_1.name)) + .xmlFile(spark, getTestResourcePath(resDir + "cars-iso-8859-1.xml")) + assert(dataFrame.select("year").collect().length === 3) + + val results = dataFrame + .select("comment", "year") + .where(dataFrame("year") === 2012) + + assert(results.head() === Row("No comment", 2012)) + } + + test("DSL test compressed file") { + val results = spark.read + .xml(getTestResourcePath(resDir + "cars.xml.gz")) + .select("year") + .collect() + + assert(results.length === 3) + } + + test("DSL test splittable compressed file") { + val results = spark.read + .xml(getTestResourcePath(resDir + "cars.xml.bz2")) + .select("year") + .collect() + + assert(results.length === 3) + } + + test("DSL test bad charset name") { + val exception = intercept[UnsupportedCharsetException] { + spark.read + .option("charset", "1-9588-osi") + .xml(getTestResourcePath(resDir + "cars.xml")) + .select("year") + .collect() + } + assert(exception.getMessage.contains("1-9588-osi")) + } + + test("DDL test") { + spark.sql(s""" + |CREATE TEMPORARY VIEW carsTable1 + |USING org.apache.spark.sql.execution.datasources.xml + |OPTIONS (path "${getTestResourcePath(resDir + "cars.xml")}") + """.stripMargin.replaceAll("\n", " ")) + + assert(spark.sql("SELECT year FROM carsTable1").collect().length === 3) + } + + test("DDL test with alias name") { + spark.sql(s""" + |CREATE TEMPORARY VIEW carsTable2 + |USING xml + |OPTIONS (path "${getTestResourcePath(resDir + "cars.xml")}") + """.stripMargin.replaceAll("\n", " ")) + + assert(spark.sql("SELECT year FROM carsTable2").collect().length === 3) + } + + test("DSL test for parsing a malformed XML file") { + val results = new XmlReader(Map("mode" -> DropMalformedMode.name)) + .xmlFile(spark, getTestResourcePath(resDir + "cars-malformed.xml")) + + assert(results.count() === 1) + } + + test("DSL test for dropping malformed rows") { + val cars = new XmlReader(Map("mode" -> DropMalformedMode.name)) + .xmlFile(spark, getTestResourcePath(resDir + "cars-malformed.xml")) + + assert(cars.count() == 1) + assert(cars.head() === Row("Chevy", "Volt", 2015)) + } + + test("DSL test for failing fast") { + val exceptionInParse = intercept[SparkException] { + new XmlReader(Map("mode" -> FailFastMode.name)) + .xmlFile(spark, getTestResourcePath(resDir + "cars-malformed.xml")) + .collect() + } + assert(exceptionInParse.getMessage.contains("Malformed line in FAILFAST mode")) + } + + test("test FAILFAST with unclosed tag") { + val exceptionInParse = intercept[SparkException] { + spark.read + .option("rowTag", "book") + .option("mode", "FAILFAST") + .xml(getTestResourcePath(resDir + "unclosed_tag.xml")) + .show() + } + assert(exceptionInParse.getMessage.contains("Malformed line in FAILFAST mode")) + } + + test("DSL test for permissive mode for corrupt records") { + val carsDf = new XmlReader(Map( + "mode" -> PermissiveMode.name, + "columnNameOfCorruptRecord" -> "_malformed_records")) + .xmlFile(spark, getTestResourcePath(resDir + "cars-malformed.xml")) + val cars = carsDf.collect() + assert(cars.length === 3) + + val malformedRowOne = carsDf.select("_malformed_records").first().get(0).toString + val malformedRowTwo = carsDf.select("_malformed_records").take(2).last.get(0).toString + val expectedMalformedRowOne = "2012Tesla>S" + + "No comment" + val expectedMalformedRowTwo = "FordE350model>" + + "Go get one now they are going fast" + + assert(malformedRowOne.replaceAll("\\s", "") === expectedMalformedRowOne.replaceAll("\\s", "")) + assert(malformedRowTwo.replaceAll("\\s", "") === expectedMalformedRowTwo.replaceAll("\\s", "")) + assert(cars(2)(0) === null) + assert(cars(0).toSeq.takeRight(3) === Seq(null, null, null)) + assert(cars(1).toSeq.takeRight(3) === Seq(null, null, null)) + assert(cars(2).toSeq.takeRight(3) === Seq("Chevy", "Volt", 2015)) + } + + test("DSL test with empty file and known schema") { + val results = new XmlReader(buildSchema(field("column", StringType, false))) + .xmlFile(spark, getTestResourcePath(resDir + "empty.xml")) + .count() + + assert(results === 0) + } + + test("DSL test with poorly formatted file and string schema") { + val schema = buildSchema( + field("color"), + field("year"), + field("make"), + field("model"), + field("comment")) + val results = new XmlReader(schema) + .xmlFile(spark, getTestResourcePath(resDir + "cars-unbalanced-elements.xml")) + .count() + + assert(results === 3) + } + + test("DDL test with empty file") { + spark.sql(s""" + |CREATE TEMPORARY VIEW carsTable3 + |(year double, make string, model string, comments string, grp string) + |USING org.apache.spark.sql.execution.datasources.xml + |OPTIONS (path "${getTestResourcePath(resDir + "empty.xml")}") + """.stripMargin.replaceAll("\n", " ")) + + assert(spark.sql("SELECT count(*) FROM carsTable3").collect().head(0) === 0) + } + + test("SQL test insert overwrite") { + val tempPath = getEmptyTempDir() + spark.sql(s""" + |CREATE TEMPORARY VIEW booksTableIO + |USING org.apache.spark.sql.execution.datasources.xml + |OPTIONS (path "${getTestResourcePath(resDir + "books.xml")}", rowTag "book") + """.stripMargin.replaceAll("\n", " ")) + spark.sql(s""" + |CREATE TEMPORARY VIEW booksTableEmpty + |(author string, description string, genre string, + |id string, price double, publish_date string, title string) + |USING org.apache.spark.sql.execution.datasources.xml + |OPTIONS (path "$tempPath") + """.stripMargin.replaceAll("\n", " ")) + + assert(spark.sql("SELECT * FROM booksTableIO").collect().length === 12) + assert(spark.sql("SELECT * FROM booksTableEmpty").collect().isEmpty) + + spark.sql( + s""" + |INSERT OVERWRITE TABLE booksTableEmpty + |SELECT * FROM booksTableIO + """.stripMargin.replaceAll("\n", " ")) + assert(spark.sql("SELECT * FROM booksTableEmpty").collect().length == 12) + } + + test("DSL save with gzip compression codec") { + val copyFilePath = getEmptyTempDir().resolve("cars-copy.xml") + + val cars = spark.read.xml(getTestResourcePath(resDir + "cars.xml")) + cars.write + .mode(SaveMode.Overwrite) + .options(Map("codec" -> classOf[GzipCodec].getName)) + .xml(copyFilePath.toString) + // Check that the part file has a .gz extension + assert(Files.exists(copyFilePath.resolve("part-00000.gz"))) + + val carsCopy = spark.read.xml(copyFilePath.toString) + + assert(carsCopy.count() === cars.count()) + assert(carsCopy.collect().map(_.toString).toSet === cars.collect().map(_.toString).toSet) + } + + test("DSL save with gzip compression codec by shorten name") { + val copyFilePath = getEmptyTempDir().resolve("cars-copy.xml") + + val cars = spark.read.xml(getTestResourcePath(resDir + "cars.xml")) + cars.write + .mode(SaveMode.Overwrite) + .options(Map("compression" -> "gZiP")) + .xml(copyFilePath.toString) + + // Check that the part file has a .gz extension + assert(Files.exists(copyFilePath.resolve("part-00000.gz"))) + + val carsCopy = spark.read.xml(copyFilePath.toString) + + assert(carsCopy.count() === cars.count()) + assert(carsCopy.collect().map(_.toString).toSet === cars.collect().map(_.toString).toSet) + } + + test("DSL save") { + val copyFilePath = getEmptyTempDir().resolve("books-copy.xml") + + val books = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated.xml")) + books.write + .options(Map("rootTag" -> "books", "rowTag" -> "book")) + .xml(copyFilePath.toString) + + val booksCopy = spark.read + .option("rowTag", "book") + .xml(copyFilePath.toString) + assert(booksCopy.count() === books.count()) + assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet) + } + + test("DSL save with declaration") { + val copyFilePath1 = getEmptyTempDir().resolve("books-copy.xml") + + val books = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated.xml")) + + books.write + .options(Map("rootTag" -> "books", "rowTag" -> "book", "declaration" -> "")) + .xml(copyFilePath1.toString) + + assert(getLines(copyFilePath1.resolve("part-00000")).head === "") + + val copyFilePath2 = getEmptyTempDir().resolve("books-copy.xml") + + books.write + .options(Map("rootTag" -> "books", "rowTag" -> "book")) + .xml(copyFilePath2.toString) + + assert(getLines(copyFilePath2.resolve("part-00000")).head === + "") + } + + test("DSL save with item") { + val tempPath = getEmptyTempDir().resolve("items-temp.xml") + val items = spark.createDataFrame(Seq(Tuple1(Array(Array(3, 4))))).toDF("thing").repartition(1) + items.write.option("arrayElementName", "foo").xml(tempPath.toString) + assert(getLines(tempPath.resolve("part-00000")).count(_.contains("")) === 2) + } + + test("DSL save with nullValue and treatEmptyValuesAsNulls") { + val copyFilePath = getEmptyTempDir().resolve("books-copy.xml") + + val books = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated.xml")) + books.write + .options(Map("rootTag" -> "books", "rowTag" -> "book", "nullValue" -> "")) + .xml(copyFilePath.toString) + + val booksCopy = spark.read + .option("rowTag", "book") + .option("treatEmptyValuesAsNulls", "true") + .xml(copyFilePath.toString) + + assert(booksCopy.count() === books.count()) + assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet) + } + + test("Write values properly as given to valueTag even if it starts with attributePrefix") { + val copyFilePath = getEmptyTempDir().resolve("books-copy.xml") + + val rootTag = "catalog" + val books = spark.read + .option("valueTag", "#VALUE") + .option("attributePrefix", "#") + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-attributes-in-no-child.xml")) + + books.write + .option("valueTag", "#VALUE") + .option("attributePrefix", "#") + .option("rootTag", rootTag) + .option("rowTag", "book") + .xml(copyFilePath.toString) + + val booksCopy = spark.read + .option("valueTag", "#VALUE") + .option("attributePrefix", "_") + .option("rowTag", "book") + .xml(copyFilePath.toString) + + assert(booksCopy.count() === books.count()) + assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet) + } + + test("DSL save dataframe not read from a XML file") { + val copyFilePath = getEmptyTempDir().resolve("data-copy.xml") + + val schema = buildSchema(array("a", ArrayType(StringType))) + val data = spark.sparkContext.parallelize( + List(List(List("aa", "bb"), List("aa", "bb")))).map(Row(_)) + val df = spark.createDataFrame(data, schema) + df.write.xml(copyFilePath.toString) + + // When [[ArrayType]] has [[ArrayType]] as elements, it is confusing what is the element + // name for XML file. Now, it is "item" by default. So, "item" field is additionally added + // to wrap the element. + val schemaCopy = buildSchema( + structArray("a", + field(XmlOptions.DEFAULT_ARRAY_ELEMENT_NAME, ArrayType(StringType)))) + val dfCopy = spark.read.xml(copyFilePath.toString) + + assert(dfCopy.count() === df.count()) + assert(dfCopy.schema === schemaCopy) + } + + test("DSL save dataframe with data types correctly") { + val copyFilePath = getEmptyTempDir().resolve("data-copy.xml") + + // Create the schema. + val dataTypes = Array( + StringType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 3), DecimalType(6, 5), + DateType, TimestampType, MapType(StringType, StringType)) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + field(s"col$index", dataType) + } + val schema = StructType(fields) + + val currentTZ = TimeZone.getDefault + try { + // Tests will depend on default timezone, so set it to UTC temporarily + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + // Create the data + val timestamp = "2015-01-01 00:00:00" + val date = "2015-01-01" + val row = + Row( + "aa", null, true, + 1.toByte, 1.toShort, 1, 1.toLong, + 1.toFloat, 1.toDouble, Decimal(1, 25, 3), Decimal(1, 6, 5), + Date.valueOf(date), Timestamp.valueOf(timestamp), Map("a" -> "b")) + val data = spark.sparkContext.parallelize(Seq(row)) + + val df = spark.createDataFrame(data, schema) + df.write.xml(copyFilePath.toString) + + val dfCopy = new XmlReader(schema) + .xmlFile(spark, copyFilePath.toString) + + assert(dfCopy.collect() === df.collect()) + assert(dfCopy.schema === df.schema) + } finally { + TimeZone.setDefault(currentTZ) + } + } + + test("DSL test schema inferred correctly") { + val results = spark.read.option("rowTag", "book").xml(getTestResourcePath(resDir + "books.xml")) + + assert(results.schema === buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + field("publish_date", DateType), + field("title"))) + + assert(results.collect().length === 12) + } + + test("DSL test schema inferred correctly with sampling ratio") { + val results = spark.read + .option("rowTag", "book") + .option("samplingRatio", 0.5) + .xml(getTestResourcePath(resDir + "books.xml")) + + assert(results.schema === buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + field("publish_date", DateType), + field("title"))) + + assert(results.collect().length === 12) + } + + test("DSL test schema (object) inferred correctly") { + val results = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-nested-object.xml")) + + assert(results.schema === buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + struct("publish_dates", + field("publish_date", DateType)), + field("title"))) + + assert(results.collect().length === 12) + } + + test("DSL test schema (array) inferred correctly") { + val results = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-nested-array.xml")) + + assert(results.schema === buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + array("publish_date", DateType), + field("title"))) + + assert(results.collect().length === 12) + } + + test("DSL test schema (complicated) inferred correctly") { + val results = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated.xml")) + + assert(results.schema == buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + struct("genre", + field("genreid", LongType), + field("name")), + field("price", DoubleType), + struct("publish_dates", + array("publish_date", + struct( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}tag"), + field("day", LongType), + field("month", LongType), + field("year", LongType)))), + field("title"))) + + assert(results.collect().length === 3) + } + + test("DSL test parsing and inferring attribute in elements having no child element") { + // Default value. + val resultsOne = new XmlReader(Map("rowTag" -> "book")) + .xmlFile(spark, getTestResourcePath(resDir + "books-attributes-in-no-child.xml")) + + val schemaOne = buildSchema( + field("_id"), + field("author"), + struct("price", + field("_VALUE"), + field(s"_unit")), + field("publish_date", DateType), + field("title")) + + assert(resultsOne.schema === schemaOne) + assert(resultsOne.count() === 12) + + // Explicitly set + val attributePrefix = "@#" + val valueTag = "#@@value" + val resultsTwo = new XmlReader(Map( + "rowTag" -> "book", "attributePrefix" -> attributePrefix, + "valueTag" -> valueTag)) + .xmlFile(spark, getTestResourcePath(resDir + "books-attributes-in-no-child.xml")) + + val schemaTwo = buildSchema( + field(s"${attributePrefix}id"), + field("author"), + struct("price", + field(valueTag), + field(s"${attributePrefix}unit")), + field("publish_date", DateType), + field("title")) + + assert(resultsTwo.schema === schemaTwo) + assert(resultsTwo.count() === 12) + } + + test("DSL test schema (excluding tags) inferred correctly") { + val results = new XmlReader(Map("excludeAttribute" -> true, "rowTag" -> "book")) + .xmlFile(spark, getTestResourcePath(resDir + "books.xml")) + + val schema = buildSchema( + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + field("publish_date", DateType), + field("title")) + + assert(results.schema === schema) + } + + test("DSL test with custom schema") { + val schema = buildSchema( + field("make"), + field("model"), + field("comment"), + field("color"), + field("year", IntegerType)) + val results = new XmlReader(schema) + .xmlFile(spark, getTestResourcePath(resDir + "cars-unbalanced-elements.xml")) + .count() + + assert(results === 3) + } + + test("DSL test inferred schema passed through") { + val dataFrame = spark.read.xml(getTestResourcePath(resDir + "cars.xml")) + + val results = dataFrame + .select("comment", "year") + .where(dataFrame("year") === 2012) + + assert(results.head() === Row("No comment", 2012)) + } + + test("DSL test nullable fields") { + val schema = buildSchema( + field("name", StringType, false), + field("age")) + val results = new XmlReader(schema) + .xmlFile(spark, getTestResourcePath(resDir + "null-numbers.xml")) + .collect() + + assert(results(0) === Row("alice", "35")) + assert(results(1) === Row("bob", " ")) + assert(results(2) === Row("coc", "24")) + } + + test("DSL test for treating empty string as null value") { + val schema = buildSchema( + field("name", StringType, false), + field("age", IntegerType)) + val results = new XmlReader(schema, Map("treatEmptyValuesAsNulls" -> true)) + .xmlFile(spark, getTestResourcePath(resDir + "null-numbers.xml")) + .collect() + + assert(results(1) === Row("bob", null)) + } + + test("DSL test with namespaces ignored") { + val results = spark.read + .option("rowTag", "Topic") + .xml(getTestResourcePath(resDir + "topics-namespaces.xml")) + .collect() + + assert(results.length === 1) + } + + test("xs_any array matches single element") { + val schema = buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + field("price", DoubleType), + field("publish_date"), + field("xs_any")) + val results = spark.read.schema(schema).option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books.xml")) + // .select("xs_any") + .collect() + results.foreach { r => + assert(r.getString(0) != null) + } + } + + test("xs_any array matches multiple elements") { + val schema = buildSchema( + field(s"${DEFAULT_ATTRIBUTE_PREFIX}id"), + field("author"), + field("description"), + field("genre"), + array("xs_any", StringType)) + val results = spark.read.schema(schema).option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books.xml")) + .collect() + results.foreach { r => + assert(r.getAs[Seq[String]]("xs_any").size === 3) + } + } + + test("Missing nested struct represented as Row of nulls instead of null") { + val result = spark.read + .option("rowTag", "item") + .xml(getTestResourcePath(resDir + "null-nested-struct.xml")) + .select("b.es") + .collect() + + assert(result(1).getStruct(0) !== null) + assert(result(1).getStruct(0)(0) === null) + } + + test("Produces correct result for empty vs non-existent rows") { + val schema = buildSchema( + struct("b", + struct("es", + field("e"), + field("f")))) + val result = spark.read + .option("rowTag", "item") + .schema(schema) + .xml(getTestResourcePath(resDir + "null-nested-struct-2.xml")) + .collect() + + assert(result(0) === Row(Row(null))) + assert(result(1) === Row(Row(Row(null, null)))) + assert(result(2) === Row(Row(Row("E", null)))) + assert(result(3) === Row(Row(Row("E", " ")))) + assert(result(4) === Row(Row(Row("E", "")))) + } + + test("Produces correct order of columns for nested rows when user specifies a schema") { + val schema = buildSchema( + struct("c", + field("b", IntegerType), + field("a", IntegerType))) + + val result = new XmlReader(schema) + .xmlFile(spark, getTestResourcePath(resDir + "simple-nested-objects.xml")) + .select("c.a", "c.b") + .collect() + + assert(result(0) === Row(111, 222)) + } + + private[this] def testNextedElementFromFile(xmlFile: String): Unit = { + val lines = getLines(Paths.get(xmlFile.replace("file:/", "/"))).toList + val firstExpected = lines(2).trim + val lastExpected = lines(3).trim + val config = new Configuration(spark.sessionState.newHadoopConf()) + config.set(XmlInputFormat.START_TAG_KEY, "") + config.set(XmlInputFormat.END_TAG_KEY, "") + val records = spark.sparkContext.newAPIHadoopFile( + xmlFile, + classOf[XmlInputFormat], + classOf[LongWritable], + classOf[Text], + config) + val list = records.values.map(_.toString).collect().toList + assert(list.length === 2) + val firstActual = list.head + val lastActual = list.last + assert(firstActual === firstExpected) + assert(lastActual === lastExpected) + } + + test("Nested element with same name as parent delineation") { + testNextedElementFromFile(getTestResourcePath(resDir + + "nested-element-with-name-of-parent.xml")) + } + + test("Nested element including attribute with same name as parent delineation") { + testNextedElementFromFile(getTestResourcePath(resDir + + "nested-element-with-attributes-and-name-of-parent.xml")) + } + + test("Nested element with same name as parent schema inference") { + val df = new XmlReader(Map("rowTag" -> "parent")) + .xmlFile(spark, getTestResourcePath(resDir + "nested-element-with-name-of-parent.xml")) + + val schema = buildSchema( + field("child"), + struct("parent", + field("child"))) + assert(df.schema === schema) + } + + test("Skip and project currently XML files without indentation") { + val df = spark.read.xml(getTestResourcePath(resDir + "cars-no-indentation.xml")) + val results = df.select("model").collect() + val years = results.map(_(0)).toSet + assert(years === Set("S", "E350", "Volt")) + } + + test("Select correctly all child fields regardless of pushed down projection") { + val results = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated.xml")) + .selectExpr("publish_dates") + .collect() + results.foreach { row => + // All nested fields should not have nulls but arrays. + assert(!row.anyNull) + } + } + + test("Empty string not allowed for rowTag, attributePrefix and valueTag.") { + val messageOne = intercept[IllegalArgumentException] { + spark.read.option("rowTag", "").xml(getTestResourcePath(resDir + "cars.xml")) + }.getMessage + assert(messageOne === "requirement failed: 'rowTag' option should not be empty string.") + + val messageThree = intercept[IllegalArgumentException] { + spark.read.option("valueTag", "").xml(getTestResourcePath(resDir + "cars.xml")) + }.getMessage + assert(messageThree === "requirement failed: 'valueTag' option should not be empty string.") + } + + test("'rowTag' and 'rootTag' should not include angle brackets") { + val messageOne = intercept[IllegalArgumentException] { + spark.read.option("rowTag", "ROW>").xml(getTestResourcePath(resDir + "cars.xml")) + }.getMessage + assert(messageOne === "requirement failed: 'rowTag' should not include angle brackets") + + val messageTwo = intercept[IllegalArgumentException] { + spark.read.option("rowTag", "").xml(getTestResourcePath(resDir + "cars.xml")) + }.getMessage + assert(messageThree === "requirement failed: 'rootTag' should not include angle brackets") + + val messageFour = intercept[IllegalArgumentException] { + spark.read.option("rootTag", " true, "rowTag" -> "person")) + .xmlFile(spark, getTestResourcePath(resDir + "ages-with-spaces.xml")) + .collect() + val attrValOne = results(0).getStruct(0)(1) + val attrValTwo = results(1).getStruct(0)(0) + assert(attrValOne.toString === "1990-02-24") + assert(attrValTwo === 30) + assert(results.length === 3) + } + + test("DSL test with malformed attributes") { + val results = new XmlReader(Map("mode" -> DropMalformedMode.name, "rowTag" -> "book")) + .xmlFile(spark, getTestResourcePath(resDir + "books-malformed-attributes.xml")) + .collect() + + assert(results.length === 2) + assert(results(0)(0) === "bk111") + assert(results(1)(0) === "bk112") + } + + test("read utf-8 encoded file with empty tag") { + val df = spark.read + .option("excludeAttribute", "false") + .option("rowTag", "House") + .xml(getTestResourcePath(resDir + "fias_house.xml")) + + assert(df.collect().length === 37) + assert(df.select().where("_HOUSEID is null").count() == 0) + } + + test("attributes start with new line") { + val schema = buildSchema( + field("_schemaLocation"), + field("_xmlns"), + field("_xsi"), + field("body"), + field("from"), + field("heading"), + field("to")) + + val rowsCount = 1 + + Seq("attributesStartWithNewLine.xml", + "attributesStartWithNewLineCR.xml", + "attributesStartWithNewLineLF.xml").foreach { file => + val df = spark.read + .option("ignoreNamespace", "true") + .option("excludeAttribute", "false") + .option("rowTag", "note") + .xml(getTestResourcePath(resDir + file)) + assert(df.schema === schema) + assert(df.count() === rowsCount) + } + } + + test("Produces correct result for a row with a self closing tag inside") { + val schema = buildSchema( + field("non-empty-tag", IntegerType), + field("self-closing-tag", IntegerType)) + + val result = new XmlReader(schema) + .xmlFile(spark, getTestResourcePath(resDir + "self-closing-tag.xml")) + .collect() + + assert(result(0) === Row(1, null)) + } + + test("DSL save with null attributes") { + val copyFilePath = getEmptyTempDir().resolve("books-copy.xml") + + val books = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "books-complicated-null-attribute.xml")) + books.write + .options(Map("rootTag" -> "books", "rowTag" -> "book")) + .xml(copyFilePath.toString) + + val booksCopy = spark.read + .option("rowTag", "book") + .xml(copyFilePath.toString) + assert(booksCopy.count() === books.count()) + assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet) + } + + test("DSL test nulls out invalid values when set to permissive and given explicit schema") { + val schema = buildSchema( + struct("integer_value", + field("_VALUE", IntegerType), + field("_int", IntegerType)), + struct("long_value", + field("_VALUE", LongType), + field("_int", StringType)), + field("float_value", FloatType), + field("double_value", DoubleType), + field("boolean_value", BooleanType), + field("string_value"), array("integer_array", IntegerType), + field("integer_map", MapType(StringType, IntegerType)), + field("_malformed_records", StringType)) + val results = spark.read + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", "_malformed_records") + .schema(schema) + .xml(getTestResourcePath(resDir + "datatypes-valid-and-invalid.xml")) + + assert(results.schema === schema) + + val Array(valid, invalid) = results.take(2) + + assert(valid.toSeq.toArray.take(schema.length - 1) === + Array(Row(10, 10), Row(10, "Ten"), 10.0, 10.0, true, + "Ten", Array(1, 2), Map("a" -> 123, "b" -> 345))) + assert(invalid.toSeq.toArray.take(schema.length - 1) === + Array(null, null, null, null, null, + "Ten", Array(2), null)) + + assert(valid.toSeq.toArray.last === null) + assert(invalid.toSeq.toArray.last.toString.contains( + Ten.toString)) + } + + test("empty string to null and back") { + val fruit = spark.read + .option("rowTag", "row") + .option("nullValue", "") + .xml(getTestResourcePath(resDir + "null-empty-string.xml")) + assert(fruit.head().getAs[String]("color") === null) + } + + test("test all string data type infer strategy") { + val text = spark.read + .option("rowTag", "ROW") + .option("inferSchema", "false") + .xml(getTestResourcePath(resDir + "textColumn.xml")) + assert(text.head().getAs[String]("col1") === "00010") + + } + + test("test default data type infer strategy") { + val default = spark.read + .option("rowTag", "ROW") + .option("inferSchema", "true") + .xml(getTestResourcePath(resDir + "textColumn.xml")) + assert(default.head().getAs[Int]("col1") === 10) + } + + test("test XML with processing instruction") { + val processingDF = spark.read + .option("rowTag", "foo") + .option("inferSchema", "true") + .xml(getTestResourcePath(resDir + "processing.xml")) + assert(processingDF.count() === 1) + } + + test("test mixed text and element children") { + val mixedDF = spark.read + .option("rowTag", "root") + .option("inferSchema", true) + .xml(getTestResourcePath(resDir + "mixed_children.xml")) + val mixedRow = mixedDF.head() + assert(mixedRow.getAs[Row](0).toSeq === Seq(" lorem ")) + assert(mixedRow.getString(1) === " ipsum ") + } + + test("test mixed text and complex element children") { + val mixedDF = spark.read + .option("rowTag", "root") + .option("inferSchema", true) + .xml(getTestResourcePath(resDir + "mixed_children_2.xml")) + assert(mixedDF.select("foo.bar").head().getString(0) === " lorem ") + assert(mixedDF.select("foo.baz.bing").head().getLong(0) === 2) + assert(mixedDF.select("missing").head().getString(0) === " ipsum ") + } + + test("test XSD validation") { + val basketDF = spark.read + .option("rowTag", "basket") + .option("inferSchema", true) + .option("rowValidationXSDPath", getTestResourcePath(resDir + "basket.xsd") + .replace("file:/", "/")) + .xml(getTestResourcePath(resDir + "basket.xml")) + // Mostly checking it doesn't fail + assert(basketDF.selectExpr("entry[0].key").head().getLong(0) === 9027) + } + + test("test XSD validation with validation error") { + val basketDF = spark.read + .option("rowTag", "basket") + .option("inferSchema", true) + .option("rowValidationXSDPath", getTestResourcePath(resDir + "basket.xsd") + .replace("file:/", "/")) + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", "_malformed_records") + .xml(getTestResourcePath(resDir + "basket_invalid.xml")) + assert(basketDF.select("_malformed_records").head().getString(0).startsWith("")) + } + + test("test XSD validation with addFile() with validation error") { + spark.sparkContext.addFile(getTestResourcePath(resDir + "basket.xsd")) + val basketDF = spark.read + .option("rowTag", "basket") + .option("inferSchema", true) + .option("rowValidationXSDPath", "basket.xsd") + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", "_malformed_records") + .xml(getTestResourcePath(resDir + "basket_invalid.xml")) + assert(basketDF.select("_malformed_records").head().getString(0).startsWith("")) + } + + test("test xmlRdd") { + val data = Seq( + "2012TeslaSNo comment", + "1997FordE350Get one", + "2015ChevyVoltNo") + val rdd = spark.sparkContext.parallelize(data) + assert(new XmlReader().xmlRdd(spark, rdd).collect().length === 3) + } + + import testImplicits._ + test("from_xml basic test") { + val xmlData = + """14ft3 + | dave guy + | + """.stripMargin + val df = Seq((8, xmlData)).toDF("number", "payload") + val xmlSchema = schema_of_xml_df(df.select("payload")) + val expectedSchema = df.schema.add("decoded", xmlSchema) + val result = df.withColumn("decoded", from_xml(df.col("payload"), xmlSchema)) + + assert(expectedSchema === result.schema) + assert(result.select("decoded.pid").head().getString(0) === "14ft3") + assert(result.select("decoded._foo").head().getString(0) === "bar") + } + + + test("from_xml array basic test") { + val xmlData = Array( + "14ft3dave guy", + "12345other guy") + val df = Seq((8, xmlData)).toDF("number", "payload") + val xmlSchema = schema_of_xml_array(df.select("payload").as[Array[String]]) + val expectedSchema = df.schema.add("decoded", xmlSchema) + val result = df.withColumn("decoded", from_xml(df.col("payload"), xmlSchema)) + assert(expectedSchema === result.schema) + // TBD: Following asserts fail when SharedSparkSession was used instead of SQLTestUtils + // Disabling them for now + // assert(result.selectExpr("decoded[0].pid").head().getString(0) === "14ft3") + // assert(result.selectExpr("decoded[1].pid").head().getString(0) === "12345") + } + + test("from_xml error test") { + // XML contains error + val xmlData = + """14ft3 + | dave guy + | + """.stripMargin + val df = spark.createDataFrame(Seq((8, xmlData))).toDF("number", "payload") + val xmlSchema = schema_of_xml_df(df.select("payload")) + val result = df.withColumn("decoded", from_xml(df.col("payload"), xmlSchema)) + assert(result.select("decoded._corrupt_record").head().getString(0).nonEmpty) + } + + test("from_xml_string basic test") { + val xmlData = + """14ft3 + | dave guy + | + """.stripMargin + val df = spark.createDataFrame(Seq((8, xmlData))).toDF("number", "payload") + val xmlSchema = schema_of_xml_df(df.select("payload")) + val result = from_xml_string(xmlData, xmlSchema) + + assert(result.getString(0) === "bar") + assert(result.getString(1) === "dave guy") + assert(result.getString(2) === "14ft3") + } + + test("from_xml with PERMISSIVE parse mode with no corrupt col schema") { + // XML contains error + val xmlData = + """14ft3 + | dave guy + | + """.stripMargin + val xmlDataNoError = + """ + | dave guy + | + """.stripMargin + val dfNoError = spark.createDataFrame(Seq((8, xmlDataNoError))).toDF("number", "payload") + val xmlSchema = schema_of_xml_df(dfNoError.select("payload")) + val df = spark.createDataFrame(Seq((8, xmlData))).toDF("number", "payload") + val result = df.withColumn("decoded", from_xml(df.col("payload"), xmlSchema)) + assert(result.select("decoded").head().get(0) === null) + } + + test("decimals with scale greater than precision") { + val spark = this.spark; + import spark.implicits._ + val schema = buildSchema(field("Number", DecimalType(7, 4))) + val outputDF = Seq("0.0000", "0.01") + .map { n => s" $n " } + .toDF("xml") + .withColumn("parsed", from_xml($"xml", schema, Map("rowTag" -> "Row"))) + .select("parsed.Number") + + val results = outputDF.collect() + assert(results(0).getAs[java.math.BigDecimal](0).toString === "0.0000") + assert(results(1).getAs[java.math.BigDecimal](0).toString === "0.0100") + } + + test("double field encounters whitespace-only value") { + val schema = buildSchema(struct("Book", field("Price", DoubleType)), field("_corrupt_record")) + val whitespaceDF = spark.read + .option("rowTag", "Books") + .schema(schema) + .xml(getTestResourcePath(resDir + "whitespace_error.xml")) + + assert(whitespaceDF.count() === 1) + assert(whitespaceDF.take(1).head.getAs[String]("_corrupt_record") !== null) + } + + test("struct with only attributes and no value tag does not crash") { + val schema = buildSchema(struct("book", field("_id", StringType)), field("_corrupt_record")) + val booksDF = spark.read + .option("rowTag", "book") + .schema(schema) + .xml(getTestResourcePath(resDir + "books.xml")) + + assert(booksDF.count() === 12) + } + + test("XML in String field preserves attributes") { + val schema = buildSchema(field("ROW")) + val result = spark.read + .option("rowTag", "ROWSET") + .schema(schema) + .xml(getTestResourcePath(resDir + "cars-attribute.xml")) + .collect() + assert(result.head.getString(0).contains("No")) + } + + test("rootTag with simple attributes") { + val xmlPath = getEmptyTempDir().resolve("simple_attributes") + val df = spark.createDataFrame(Seq((42, "foo"))).toDF("number", "value").repartition(1) + df.write. + option("rootTag", "root foo='bar' bing=\"baz\""). + option("declaration", ""). + xml(xmlPath.toString) + + val xmlFile = + Files.list(xmlPath).iterator.asScala.filter(_.getFileName.toString.startsWith("part-")).next() + val firstLine = getLines(xmlFile).head + assert(firstLine === "") + } + + test("test ignoreNamespace") { + val results = spark.read + .option("rowTag", "book") + .option("ignoreNamespace", true) + .xml(getTestResourcePath(resDir + "books-namespaces.xml")) + assert(results.filter("author IS NOT NULL").count() === 3) + assert(results.filter("_id IS NOT NULL").count() === 3) + } + + test("MapType field with attributes") { + val schema = buildSchema( + field("_startTime"), + field("_interval"), + field("PMTarget", MapType(StringType, StringType))) + val df = spark.read.option("rowTag", "PMSetup"). + schema(schema). + xml(getTestResourcePath(resDir + "map-attribute.xml")). + select("PMTarget") + val map = df.collect().head.getAs[Map[String, String]](0) + assert(map.contains("_measurementType")) + assert(map.contains("M1")) + assert(map.contains("M2")) + } + + test("StructType with missing optional StructType child") { + val df = spark.read.option("rowTag", "Foo") + .xml(getTestResourcePath(resDir + "struct_with_optional_child.xml")) + assert(df.selectExpr("SIZE(Bar)").collect().head.getInt(0) === 2) + } + + test("Manual schema with corrupt record field works on permissive mode failure") { + // See issue #517 + val schema = StructType(List( + StructField("_id", StringType), + StructField("_space", StringType), + StructField("c2", DoubleType), + StructField("c3", StringType), + StructField("c4", StringType), + StructField("c5", StringType), + StructField("c6", StringType), + StructField("c7", StringType), + StructField("c8", StringType), + StructField("c9", DoubleType), + StructField("c11", DoubleType), + StructField("c20", ArrayType(StructType(List( + StructField("_VALUE", StringType), + StructField("_m", IntegerType))) + )), + StructField("c46", StringType), + StructField("c76", StringType), + StructField("c78", StringType), + StructField("c85", DoubleType), + StructField("c93", StringType), + StructField("c95", StringType), + StructField("c99", ArrayType(StructType(List( + StructField("_VALUE", StringType), + StructField("_m", IntegerType))) + )), + StructField("c100", ArrayType(StructType(List( + StructField("_VALUE", StringType), + StructField("_m", IntegerType))) + )), + StructField("c108", StringType), + StructField("c192", DoubleType), + StructField("c193", StringType), + StructField("c194", StringType), + StructField("c195", StringType), + StructField("c196", StringType), + StructField("c197", DoubleType), + StructField("_corrupt_record", StringType))) + + val df = spark.read + .option("inferSchema", false) + .option("rowTag", "row") + .schema(schema) + .xml(getTestResourcePath(resDir + "manual_schema_corrupt_record.xml")) + + // Assert it works at all + assert(df.collect().head.getAs[String]("_corrupt_record") !== null) + } + + test("Test date parsing") { + val schema = buildSchema(field("author"), field("date", DateType), field("date2", StringType)) + val df = spark.read + .option("rowTag", "book") + .schema(schema) + .xml(getTestResourcePath(resDir + "date.xml")) + assert(df.collect().head.getAs[Date](1).toString === "2021-02-01") + } + + test("Test date type inference") { + val df = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "date.xml")) + val expectedSchema = + buildSchema(field("author"), field("date", DateType), field("date2", StringType)) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Date](1).toString === "2021-02-01") + } + + test("Test timestamp parsing") { + val schema = + buildSchema(field("author"), field("time", TimestampType), field("time2", StringType)) + val df = spark.read + .option("rowTag", "book") + .schema(schema) + .xml(getTestResourcePath(resDir + "time.xml")) + assert(df.collect().head.getAs[Timestamp](1).getTime === 1322907330000L) + } + + test("Test timestamp type inference") { + val df = spark.read + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "time.xml")) + val expectedSchema = + buildSchema( + field("author"), + field("time", TimestampType), + field("time2", StringType), + field("time3", StringType), + field("time4", StringType) + ) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Timestamp](1).getTime === 1322907330000L) + } + + test("Test dateFormat") { + val df = spark.read + .option("rowTag", "book") + .option("dateFormat", "MM-dd-yyyy") + .xml(getTestResourcePath(resDir + "date.xml")) + val expectedSchema = + buildSchema(field("author"), field("date", DateType), field("date2", DateType)) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Date](2).toString === "2021-02-01") + } + + test("Test timestampFormat") { + val df = spark.read + .option("rowTag", "book") + .option("timestampFormat", "MM-dd-yyyy HH:mm:ss z") + .xml(getTestResourcePath(resDir + "time.xml")) + val expectedSchema = + buildSchema( + field("author"), + field("time", TimestampType), + field("time2", TimestampType), + field("time3", StringType), + field("time4", StringType) + ) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Timestamp](1).getTime === 1322907330000L) + assert(df.collect().head.getAs[Timestamp](2).getTime === 1322936130000L) + } + + test("Test custom timestampFormat without timezone") { + val df = spark.read + .option("rowTag", "book") + .option("timestampFormat", "yyyy/MM/dd HH:mm:ss") + .xml(getTestResourcePath(resDir + "time.xml")) + val expectedSchema = + buildSchema( + field("author"), + field("time", TimestampType), + field("time2", StringType), + field("time3", TimestampType), + field("time4", StringType) + ) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Timestamp](1).getTime === 1322907330000L) + assert(df.collect().head.getAs[Timestamp](3).getTime === 1322892930000L) + } + + test("Test custom timestampFormat with offset") { + val df = spark.read + .option("rowTag", "book") + .option("timestampFormat", "yyyy/MM/dd HH:mm:ss xx") + .xml(getTestResourcePath(resDir + "time.xml")) + val expectedSchema = + buildSchema( + field("author"), + field("time", TimestampType), + field("time2", StringType), + field("time3", StringType), + field("time4", TimestampType) + ) + assert(df.schema === expectedSchema) + assert(df.collect().head.getAs[Timestamp](1).getTime === 1322907330000L) + assert(df.collect().head.getAs[Timestamp](4).getTime === 1322892930000L) + } + + test("Test null number type is null not 0.0") { + val schema = buildSchema( + struct("Header", + field("_Name"), field("_SequenceNumber", LongType)), + structArray("T", + field("_Number", LongType), field("_VALUE", DoubleType), field("_Volume", DoubleType))) + + val df = spark.read.option("rowTag", "TEST") + .option("nullValue", "") + .schema(schema) + .xml(getTestResourcePath(resDir + "null-numbers-2.xml")) + .select(explode(column("T"))) + + assert(df.collect()(1).getStruct(0).get(2) === null) + } + + test("read multiple xml files in parallel") { + val failedAgesSet = mutable.Set[Long]() + val threads_ages = (1 to 10).map { i => + new Thread { + override def run(): Unit = { + val df = spark.read.option("rowTag", "person").format("xml") + .load(getTestResourcePath(resDir + "ages.xml")) + if (df.schema.fields.isEmpty) { + failedAgesSet.add(i) + } + } + } + } + + val failedBooksSet = mutable.Set[Long]() + val threads_books = (11 to 20).map { i => + new Thread { + override def run(): Unit = { + val df = spark.read.option("rowTag", "book").format("xml") + .load(getTestResourcePath(resDir + "books.xml")) + if (df.schema.fields.isEmpty) { + failedBooksSet.add(i) + } + } + } + } + + threads_ages.foreach(_.start()) + threads_books.foreach(_.start()) + threads_ages.foreach(_.join()) + threads_books.foreach(_.join()) + assert(failedBooksSet.isEmpty) + assert(failedAgesSet.isEmpty) + } + + test("Issue 588: Ensure fails when data is not present, with or without schema") { + intercept[InvalidInputException] { + spark.read.xml("/this/file/does/not/exist") + } + intercept[InvalidInputException] { + spark.read.schema(buildSchema(field("dummy"))).xml("/this/file/does/not/exist") + } + } + + test("Issue 614: mixed content element parsed as string in schema") { + val textResults = spark.read + .schema(buildSchema(field("text"))) + .option("rowTag", "book") + .xml(getTestResourcePath(resDir + "mixed_children_as_string.xml")) + val textHead = textResults.select("text").head().getString(0) + assert(textHead.contains( + "Lorem ipsum dolor sit amet. Ut voluptas distinctio et impedit deserunt")) + assert(textHead.contains( + "numquam incidunt cum autem temporibus.")) + + val bookResults = spark.read + .schema(buildSchema(field("book"))) + .option("rowTag", "books") + .xml(getTestResourcePath(resDir + "mixed_children_as_string.xml")) + val bookHead = bookResults.select("book").head().getString(0) + assert(bookHead.contains( + "Lorem ipsum dolor sit amet. Ut voluptas distinctio et impedit deserunt")) + assert(bookHead.contains( + "numquam incidunt cum autem temporibus.")) + } + + private def getLines(path: Path): Seq[String] = { + val source = Source.fromFile(path.toFile) + try { + source.getLines().toList + } finally { + source.close() + } + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGeneratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGeneratorSuite.scala new file mode 100644 index 0000000000000..176cfd985638a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGeneratorSuite.scala @@ -0,0 +1,78 @@ +/* + * 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.execution.datasources.xml.parsers + +import java.nio.file.Files +import java.sql.{Date, Timestamp} +import java.time.{ZonedDateTime, ZoneId} + +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +case class KnownData( + booleanDatum: Boolean, + dateDatum: Date, + decimalDatum: Decimal, + doubleDatum: Double, + integerDatum: Integer, + longDatum: Long, + stringDatum: String, + timeDatum: String, + timestampDatum: Timestamp, + nullDatum: Null +) + +final class StaxXmlGeneratorSuite extends SharedSparkSession { + test("write/read roundtrip") { + import testImplicits._ + + val dataset = Seq( + KnownData( + booleanDatum = true, + dateDatum = Date.valueOf("2016-12-18"), + decimalDatum = Decimal(54.321, 10, 3), + doubleDatum = 42.4242, + integerDatum = 17, + longDatum = 1520828868, + stringDatum = "test,breakdelimiter", + timeDatum = "12:34:56", + timestampDatum = Timestamp.from(ZonedDateTime.of(2017, 12, 20, 21, 46, 54, 0, + ZoneId.of("UTC")).toInstant), + nullDatum = null), + KnownData(booleanDatum = false, + dateDatum = Date.valueOf("2016-12-19"), + decimalDatum = Decimal(12.345, 10, 3), + doubleDatum = 21.2121, + integerDatum = 34, + longDatum = 1520828123, + stringDatum = "breakdelimiter,test", + timeDatum = "23:45:16", + timestampDatum = Timestamp.from(ZonedDateTime.of(2017, 12, 29, 17, 21, 49, 0, + ZoneId.of("America/New_York")).toInstant), + nullDatum = null) + ) + + val df = dataset.toDF().orderBy("booleanDatum") + val targetFile = + Files.createTempDirectory("StaxXmlGeneratorSuite").resolve("roundtrip.xml").toString + df.write.format("xml").save(targetFile) + val newDf = + spark.read.schema(df.schema).format("xml").load(targetFile).orderBy("booleanDatum") + assert(df.collect().toSeq === newDf.collect().toSeq) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtilsSuite.scala new file mode 100644 index 0000000000000..0feadab828472 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlParserUtilsSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.execution.datasources.xml.parsers + +import java.io.StringReader +import javax.xml.stream.{XMLInputFactory, XMLStreamConstants} +import javax.xml.stream.events.Attribute + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.xml.XmlOptions + +final class StaxXmlParserUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { + + private val factory = StaxXmlParserUtils.factory + + test("Test if elements are skipped until the given event type") { + val input = 2Sam Mad Dog Smith93 + val parser = factory.createXMLEventReader(new StringReader(input.toString)) + val event = StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.END_DOCUMENT) + assert(event.isEndDocument) + } + + test("Check the end of element") { + val input = 2 + val parser = factory.createXMLEventReader(new StringReader(input.toString)) + // Skip until + StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.END_ELEMENT) + assert(StaxXmlParserUtils.checkEndElement(parser)) + } + + test("Convert attributes to a map with keys and values") { + val input = + val parser = factory.createXMLEventReader(new StringReader(input.toString)) + val event = + StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.START_ELEMENT) + val attributes = + event.asStartElement().getAttributes.asScala.map(_.asInstanceOf[Attribute]).toArray + val valuesMap = + StaxXmlParserUtils.convertAttributesToValuesMap(attributes, new XmlOptions()) + assert(valuesMap === Map(s"${XmlOptions.DEFAULT_ATTRIBUTE_PREFIX}id" -> "2")) + } + + test("Convert current structure to string") { + val input = 2 + Sam Mad Dog Smith19 + val parser = factory.createXMLEventReader(new StringReader(input.toString)) + // Skip until + StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.END_ELEMENT) + val xmlString = StaxXmlParserUtils.currentStructureAsString(parser) + val expected = + Sam Mad Dog Smith19 + assert(xmlString === expected.toString()) + } + + test("Skip XML children") { + val input = + Sam Mad Dog Smith1 + 922 + val parser = factory.createXMLEventReader(new StringReader(input.toString)) + // We assume here it's reading the value within `id` field. + StaxXmlParserUtils.skipUntil(parser, XMLStreamConstants.CHARACTERS) + StaxXmlParserUtils.skipChildren(parser) + assert(parser.nextEvent().asEndElement().getName.getLocalPart === "info") + parser.next() + StaxXmlParserUtils.skipChildren(parser) + assert(parser.nextEvent().asEndElement().getName.getLocalPart === "abc") + parser.next() + StaxXmlParserUtils.skipChildren(parser) + assert(parser.nextEvent().asEndElement().getName.getLocalPart === "test") + } + + test("XML Input Factory disables DTD parsing") { + assert(factory.getProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES) === false) + assert(factory.getProperty(XMLInputFactory.SUPPORT_DTD) === false) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCastSuite.scala new file mode 100644 index 0000000000000..45f6f820cbbc5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/TypeCastSuite.scala @@ -0,0 +1,236 @@ +/* + * 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.execution.datasources.xml.util + +import java.math.BigDecimal +import java.sql.{Date, Timestamp} +import java.time.{ZonedDateTime, ZoneId} +import java.util.Locale + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.xml.XmlOptions +import org.apache.spark.sql.types._ + +final class TypeCastSuite extends SparkFunSuite { + + test("Can parse decimal type values") { + val options = new XmlOptions() + val stringValues = Seq("10.05", "1,000.01", "158,058,049.001") + val decimalValues = Seq(10.05, 1000.01, 158058049.001) + val decimalType = DecimalType.SYSTEM_DEFAULT + + stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => + val dt = new BigDecimal(decimalVal.toString) + assert(TypeCast.castTo(strVal, decimalType, options) === + Decimal(dt, dt.precision(), dt.scale())) + } + } + + test("Nullable types are handled") { + val options = new XmlOptions(Map("nullValue" -> "-")) + for (t <- Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + BooleanType, TimestampType, DateType, StringType)) { + assert(TypeCast.castTo("-", t, options) === null) + } + } + + test("String type should always return the same as the input") { + val options = new XmlOptions() + assert(TypeCast.castTo("", StringType, options) === "") + } + + test("Types are cast correctly") { + val options = new XmlOptions() + assert(TypeCast.castTo("10", ByteType, options) === 10) + assert(TypeCast.castTo("10", ShortType, options) === 10) + assert(TypeCast.castTo("10", IntegerType, options) === 10) + assert(TypeCast.castTo("10", LongType, options) === 10) + assert(TypeCast.castTo("1.00", FloatType, options) === 1.0) + assert(TypeCast.castTo("1.00", DoubleType, options) === 1.0) + assert(TypeCast.castTo("true", BooleanType, options) === true) + assert(TypeCast.castTo("1", BooleanType, options) === true) + assert(TypeCast.castTo("false", BooleanType, options) === false) + assert(TypeCast.castTo("0", BooleanType, options) === false) + assert( + TypeCast.castTo("2002-05-30 21:46:54", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("UTC")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("UTC")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54.1234", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 123400000, ZoneId.of("UTC")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54Z", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("UTC")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54-06:00", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("-06:00")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54+06:00", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("+06:00")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54.1234Z", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 123400000, ZoneId.of("UTC")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54.1234-06:00", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 123400000, ZoneId.of("-06:00")) + .toInstant() + ) + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54.1234+06:00", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 123400000, ZoneId.of("+06:00")) + .toInstant() + ) + ) + assert(TypeCast.castTo("2002-09-24", DateType, options) === Date.valueOf("2002-09-24")) + assert(TypeCast.castTo("2002-09-24Z", DateType, options) === Date.valueOf("2002-09-24")) + assert(TypeCast.castTo("2002-09-24-06:00", DateType, options) === Date.valueOf("2002-09-24")) + assert(TypeCast.castTo("2002-09-24+06:00", DateType, options) === Date.valueOf("2002-09-24")) + } + + test("Types with sign are cast correctly") { + val options = new XmlOptions() + assert(TypeCast.signSafeToInt("+10", options) === 10) + assert(TypeCast.signSafeToLong("-10", options) === -10) + assert(TypeCast.signSafeToFloat("1.00", options) === 1.0) + assert(TypeCast.signSafeToDouble("-1.00", options) === -1.0) + } + + test("Types with sign are checked correctly") { + assert(TypeCast.isBoolean("true")) + assert(TypeCast.isInteger("10")) + assert(TypeCast.isLong("10")) + assert(TypeCast.isDouble("+10.1")) + assert(!TypeCast.isDouble("8E9D")) + assert(!TypeCast.isDouble("8E9F")) + val timestamp = "2015-01-01 00:00:00" + assert(TypeCast.isTimestamp(timestamp, new XmlOptions())) + } + + test("Float and Double Types are cast correctly with Locale") { + val options = new XmlOptions() + val defaultLocale = Locale.getDefault + try { + Locale.setDefault(Locale.FRANCE) + assert(TypeCast.castTo("1,00", FloatType, options) === 1.0) + assert(TypeCast.castTo("1,00", DoubleType, options) === 1.0) + } finally { + Locale.setDefault(defaultLocale) + } + } + + test("Parsing built-in timestamp formatters") { + val options = XmlOptions(Map()) + val expectedResult = Timestamp.from( + ZonedDateTime.of(2002, 5, 30, 21, 46, 54, 0, ZoneId.of("UTC")) + .toInstant + ) + assert( + TypeCast.castTo("2002-05-30 21:46:54", TimestampType, options) === expectedResult + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54", TimestampType, options) === expectedResult + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54+00:00", TimestampType, options) === expectedResult + ) + assert( + TypeCast.castTo("2002-05-30T21:46:54.0000Z", TimestampType, options) === expectedResult + ) + } + + test("Custom timestamp format is used to parse correctly") { + var options = XmlOptions(Map("timestampFormat" -> "MM-dd-yyyy HH:mm:ss", "timezone" -> "UTC")) + assert( + TypeCast.castTo("12-03-2011 10:15:30", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2011, 12, 3, 10, 15, 30, 0, ZoneId.of("UTC")) + .toInstant + ) + ) + + options = XmlOptions(Map("timestampFormat" -> "yyyy/MM/dd HH:mm:ss", "timezone" -> "UTC")) + assert( + TypeCast.castTo("2011/12/03 10:15:30", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2011, 12, 3, 10, 15, 30, 0, ZoneId.of("UTC")) + .toInstant + ) + ) + + options = XmlOptions(Map("timestampFormat" -> "yyyy/MM/dd HH:mm:ss", + "timezone" -> "Asia/Shanghai")) + assert( + TypeCast.castTo("2011/12/03 10:15:30", TimestampType, options) !== + Timestamp.from( + ZonedDateTime.of(2011, 12, 3, 10, 15, 30, 0, ZoneId.of("UTC")) + .toInstant + ) + ) + + options = XmlOptions(Map("timestampFormat" -> "yyyy/MM/dd HH:mm:ss", + "timezone" -> "Asia/Shanghai")) + assert( + TypeCast.castTo("2011/12/03 10:15:30", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2011, 12, 3, 10, 15, 30, 0, ZoneId.of("Asia/Shanghai")) + .toInstant + ) + ) + + options = XmlOptions(Map("timestampFormat" -> "yyyy/MM/dd HH:mm:ss")) + intercept[IllegalArgumentException]( + TypeCast.castTo("2011/12/03 10:15:30", TimestampType, options) === + Timestamp.from( + ZonedDateTime.of(2011, 12, 3, 10, 15, 30, 0, ZoneId.of("UTC")) + .toInstant + ) + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchemaSuite.scala new file mode 100644 index 0000000000000..10b14a3f6bc4a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XSDToSchemaSuite.scala @@ -0,0 +1,186 @@ +/* + * 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.execution.datasources.xml.util + +import java.nio.file.Paths + +import org.apache.spark.sql.execution.datasources.xml.TestUtils._ +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, LongType, StringType} + +class XSDToSchemaSuite extends SharedSparkSession { + + private val resDir = "test-data/xml-resources/" + + test("Basic parsing") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "basket.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("basket", + struct( + structArray("entry", + field("key"), + field("value"))), nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("Relative path parsing") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "include-example/first.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("basket", + struct( + structArray("entry", + field("key"), + field("value"))), nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("Test schema types and attributes") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "catalog.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("catalog", + struct( + field("product", + struct( + structArray("catalog_item", + field("item_number", nullable = false), + field("price", FloatType, nullable = false), + structArray("size", + structArray("color_swatch", + field("_VALUE"), + field("_image")), + field("_description")), + field("_gender")), + field("_description"), + field("_product_image")), + nullable = false)), + nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("Test xs:choice nullability") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "choice.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("el", struct(field("foo"), field("bar"), field("baz")), nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("Two root elements") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "twoelements.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema(field("bar", nullable = false), field("foo", nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("xs:any schema") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "xsany.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("root", + struct( + field("foo", + struct( + field("xs_any")), + nullable = false), + field("bar", + struct( + field("xs_any", nullable = false)), + nullable = false), + field("baz", + struct( + field("xs_any", ArrayType(StringType), nullable = false)), + nullable = false), + field("bing", + struct( + field("xs_any")), + nullable = false)), + nullable = false)) + assert(expectedSchema === parsedSchema) + } + + test("Tests xs:long type / Issue 520") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "long.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("test", + struct(field("userId", LongType, nullable = false)), nullable = false)) + assert(parsedSchema === expectedSchema) + } + + test("Test xs:decimal type with restriction[fractionalDigits]") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + + "decimal-with-restriction.xsd").replace("file:/", "/"))) + val expectedSchema = buildSchema( + field("decimal_type_3", DecimalType(12, 6), nullable = false), + field("decimal_type_1", DecimalType(38, 18), nullable = false), + field("decimal_type_2", DecimalType(38, 2), nullable = false) + ) + assert(parsedSchema === expectedSchema) + } + + test("Test ref attribute / Issue 617") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + "ref-attribute.xsd") + .replace("file:/", "/"))) + val expectedSchema = buildSchema( + field( + "book", + struct( + field("name", StringType, false), + field("author", StringType, false), + field("isbn", StringType, false) + ), + false + ), + field( + "bookList", + struct( + structArray( + "book", + field("name", StringType, false), + field("author", StringType, false), + field("isbn", StringType, false) + ) + ), + false + ) + ) + assert(parsedSchema === expectedSchema) + } + + test("Test complex content with extension element / Issue 554") { + val parsedSchema = XSDToSchema.read(Paths.get(testFile(resDir + + "complex-content-extension.xsd").replace("file:/", "/"))) + + val expectedSchema = buildSchema( + field( + "employee", + struct( + field("firstname", StringType, false), + field("lastname", StringType, false), + field("address", StringType, false), + field("city", StringType, false), + field("country", StringType, false) + ), + false + ) + ) + assert(parsedSchema === expectedSchema) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFileSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFileSuite.scala new file mode 100644 index 0000000000000..075f38153ad79 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFileSuite.scala @@ -0,0 +1,69 @@ +/* + * 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.execution.datasources.xml.util + +import java.nio.charset.{StandardCharsets, UnsupportedCharsetException} + +import org.apache.spark.sql.test.SharedSparkSession + +final class XmlFileSuite extends SharedSparkSession { + + private val resourcePrefix = "test-data/xml-resources/" + private val booksFile = testFile(resourcePrefix + "books.xml") + private val booksUnicodeInTagNameFile = testFile(resourcePrefix + "books-unicode-in-tag-name.xml") + private val booksFileTag = "book" + private val booksUnicodeFileTag = "\u66F8" // scalastyle:ignore + private val numBooks = 12 + private val numBooksUnicodeInTagName = 3 + private val fiasHouse = testFile(resourcePrefix + "fias_house.xml") + private val fiasRowTag = "House" + private val numHouses = 37 + private val utf8 = StandardCharsets.UTF_8.name + + override def beforeAll(): Unit = { + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + } + + test("read utf-8 encoded file") { + val baseRDD = XmlFile.withCharset(sparkContext, booksFile, utf8, rowTag = booksFileTag) + assert(baseRDD.count() === numBooks) + } + + test("read file with unicode chars in row tag name") { + val baseRDD = XmlFile.withCharset( + sparkContext, booksUnicodeInTagNameFile, utf8, rowTag = booksUnicodeFileTag) + assert(baseRDD.count() === numBooksUnicodeInTagName) + } + + test("read utf-8 encoded file with empty tag") { + val baseRDD = XmlFile.withCharset(sparkContext, fiasHouse, utf8, rowTag = fiasRowTag) + assert(baseRDD.count() == numHouses) + baseRDD.collect().foreach(x => assert(x.contains("/>"))) + } + + test("unsupported charset") { + val exception = intercept[UnsupportedCharsetException] { + XmlFile.withCharset(sparkContext, booksFile, "frylock", rowTag = booksFileTag).count() + } + assert(exception.getMessage.contains("frylock")) + } + +}