sandip-db commented on code in PR #42462: URL: https://github.com/apache/spark/pull/42462#discussion_r1300493066
########## sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala: ########## @@ -0,0 +1,206 @@ +/* + * 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.catalyst.expressions.xml + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, ExprUtils, NullIntolerant, TimeZoneAwareExpression, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.{ArrayData, FailFastMode, FailureSafeParser, GenericArrayData, PermissiveMode} +import org.apache.spark.sql.catalyst.xml.{StaxXmlParser, ValidatorUtil, XmlInferSchema, XmlOptions} +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +case class XmlToStructs( + schema: DataType, + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends UnaryExpression + with TimeZoneAwareExpression + with CodegenFallback + with ExpectsInputTypes + with NullIntolerant + with QueryErrorsBase { + + def this(child: Expression, schema: Expression, options: Map[String, String]) = + this( + schema = ExprUtils.evalSchemaExpr(schema), Review Comment: DataType is not fully functional. Removed the from_xml with DataType arg. ########## connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala: ########## @@ -7227,6 +7227,150 @@ object functions { */ def to_csv(e: Column): Column = to_csv(e, Collections.emptyMap()) + // scalastyle:off line.size.limit + /** + * Parses a column containing a XML string into a `StructType` with the specified schema. + * Returns `null`, in the case of an unparseable string. + * + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the XML string + * @param options + * options to control how the XML is parsed. accepts the same options and the XML data source. + * See <a href= + * "https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> Data + * Source Option</a> in the version you use. + * @group collection_funcs + * + * @since 4.0.0 + */ + // scalastyle:on line.size.limit + def from_xml(e: Column, schema: StructType, options: Map[String, String]): Column = + from_xml(e, lit(schema.toDDL), options.iterator) + + // scalastyle:off line.size.limit + + /** + * Parses a column containing a XML string into a `StructType` with the specified schema. + * Returns `null`, in the case of an unparseable string. + * + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the xml string + * @param options + * options to control how the xml is parsed. accepts the same options and the XML data source. + * See <a href= + * "https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> Data + * Source Option</a> in the version you use. + * @group collection_funcs + * + * @since 4.0.0 + */ + // scalastyle:on line.size.limit + def from_xml(e: Column, schema: DataType, options: Map[String, String]): Column = + from_xml(e, lit(schema.json), options.iterator) Review Comment: DataType is not fully functional. Removed the from_xml with DataType arg. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
