maropu commented on a change in pull request #29085: URL: https://github.com/apache/spark/pull/29085#discussion_r458596110
########## File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql ########## @@ -0,0 +1,72 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +as t1(a, b, c, d, e, f, g, h, i, j, k, l); Review comment: nit: `as` -> `AS` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala ########## @@ -56,10 +69,88 @@ trait BaseScriptTransformationExec extends UnaryExecNode { } } - def processIterator( + protected def initProc: (OutputStream, Process, InputStream, CircularBuffer) = { + val cmd = List("/bin/bash", "-c", script) + val builder = new ProcessBuilder(cmd.asJava) + + val proc = builder.start() + val inputStream = proc.getInputStream + val outputStream = proc.getOutputStream + val errorStream = proc.getErrorStream + + // In order to avoid deadlocks, we need to consume the error output of the child process. + // To avoid issues caused by large error output, we use a circular buffer to limit the amount + // of error output that we retain. See SPARK-7862 for more discussion of the deadlock / hang + // that motivates this. + val stderrBuffer = new CircularBuffer(2048) + new RedirectThread( + errorStream, + stderrBuffer, + s"Thread-${this.getClass.getSimpleName}-STDERR-Consumer").start() + (outputStream, proc, inputStream, stderrBuffer) + } + + protected def processIterator( inputIterator: Iterator[InternalRow], hadoopConf: Configuration): Iterator[InternalRow] + protected def createOutputIteratorWithoutSerde( + writerThread: BaseScriptTransformationWriterThread, + inputStream: InputStream, + proc: Process, + stderrBuffer: CircularBuffer): Iterator[InternalRow] = { + new Iterator[InternalRow] { + var curLine: String = null + val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)) + + val outputRowFormat = ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD") + val processRowWithoutSerde = if (!ioschema.schemaLess) { + prevLine: String => + new GenericInternalRow( + prevLine.split(outputRowFormat) + .zip(fieldWriters) + .map { case (data, writer) => writer(data) }) + } else { + // In schema less mode, hive default serde will choose first two output column as output + // if output column size less then 2, it will throw ArrayIndexOutOfBoundsException. + // Here we change spark's behavior same as hive's default serde + prevLine: String => + new GenericInternalRow( + prevLine.split(outputRowFormat).slice(0, 2) + .map(CatalystTypeConverters.createToCatalystConverter(StringType))) Review comment: ``` val kvWriter = CatalystTypeConverters.createToCatalystConverter(StringType) prevLine: String => new GenericInternalRow( prevLine.split(outputRowFormat).slice(0, 2) .map(kvWriter)) ``` ########## File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql ########## @@ -0,0 +1,72 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +as t1(a, b, c, d, e, f, g, h, i, j, k, l); + +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t1; + +-- with non-exist command +SELECT TRANSFORM(a) +USING 'some_non_existent_command' AS (a) +FROM t1; + +-- with non-exist file +SELECT TRANSFORM(a) +USING 'python some_non_existent_file' AS (a) +FROM t1; + +-- common supported data types between no serde and serde transform +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t1 +) tmp; + +-- handle schema less +SELECT TRANSFORM(a) +USING 'cat' +FROM t1; + +SELECT TRANSFORM(a, b) +USING 'cat' +FROM t1; + +SELECT TRANSFORM(a, b, c) +USING 'cat' +FROM t1; + +-- return null when return string incompatible (no serde) +SELECT TRANSFORM(a, b, c, d, e, f, g, h, i) +USING 'cat' as (a int, b short, c long, d byte, e float, f double, g decimal(38, 18), h date, i timestamp) +FROM VALUES +('a','','1231a','a','213.21a','213.21a','0a.21d','2000-04-01123','1997-0102 00:00:') tmp(a, b, c, d, e, f, g, h, i); + +-- SPARK-28227: transform can't run with aggregation +SELECT TRANSFORM(b, max(a), sum(f)) +USING 'cat' AS (a, b) +FROM t1 +GROUP BY b; + +-- transform use MAP +MAP a, b USING 'cat' AS (a, b) FROM t1; + +-- transform use REDUCE +REDUCE a, b USING 'cat' AS (a, b) FROM t1; + + Review comment: nit: remove blank line. ########## File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql ########## @@ -0,0 +1,72 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +as t1(a, b, c, d, e, f, g, h, i, j, k, l); + +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t1; + +-- with non-exist command +SELECT TRANSFORM(a) +USING 'some_non_existent_command' AS (a) +FROM t1; + +-- with non-exist file +SELECT TRANSFORM(a) +USING 'python some_non_existent_file' AS (a) +FROM t1; + Review comment: Could you add a test for output as strings like this? ``` -- common supported data types between no serde and serde transform SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) USING 'cat' AS ( a string, b string, c string, d... ... FROM t1 ) tmp; ``` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ########## @@ -713,13 +714,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } (Seq.empty, Option(name), props.toSeq, recordHandler) - case null => + case null if conf.getConf(CATALOG_IMPLEMENTATION).equals("hive") => // Use default (serde) format. val name = conf.getConfString("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") val props = Seq("field.delim" -> "\t") val recordHandler = Option(conf.getConfString(configKey, defaultConfigValue)) (Nil, Option(name), props, recordHandler) Review comment: Also, could we throw an exception when custom serde given without hive? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ########## @@ -713,13 +714,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } (Seq.empty, Option(name), props.toSeq, recordHandler) - case null => + case null if conf.getConf(CATALOG_IMPLEMENTATION).equals("hive") => // Use default (serde) format. val name = conf.getConfString("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") val props = Seq("field.delim" -> "\t") val recordHandler = Option(conf.getConfString(configKey, defaultConfigValue)) (Nil, Option(name), props, recordHandler) Review comment: Except for the code part above, could we move this method into `AstBuilder.withScriptIOSchema`? ########## File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala ########## @@ -206,75 +169,83 @@ class HiveScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with val query = sql( s""" - |SELECT - |TRANSFORM(a, b, c, d, e) - |USING 'python $scriptFilePath' AS (a, b, c, d, e) - |FROM v + |SELECT TRANSFORM(a, b, c, d, e) + |USING 'python ${scriptFilePath}' + |FROM v """.stripMargin) - // In Hive 1.2, the string representation of a decimal omits trailing zeroes. - // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. - val decimalToString: Column => Column = if (HiveUtils.isHive23) { - c => c.cast("string") - } else { - c => c.cast("decimal(1, 0)").cast("string") - } - checkAnswer(query, identity, df.select( - 'a.cast("string"), - 'b.cast("string"), - 'c.cast("string"), - decimalToString('d), - 'e.cast("string")).collect()) + // In hive default serde mode, if we don't define output schema, it will choose first + // two column as output schema (key: String, value: String) + checkAnswer( + query, + identity, + df.select( + 'a.cast("string").as("key"), + 'b.cast("string").as("value")).collect()) } } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (no serde)") { + test("SPARK-32106: TRANSFORM support complex data types as input and ouput type (hive serde)") { Review comment: `support` -> `supports` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala ########## @@ -87,17 +178,69 @@ trait BaseScriptTransformationExec extends UnaryExecNode { } } } + + private lazy val fieldWriters: Seq[String => Any] = output.map { attr => + val converter = CatalystTypeConverters.createToCatalystConverter(attr.dataType) + attr.dataType match { + case StringType => wrapperConvertException(data => data, converter) + case BooleanType => wrapperConvertException(data => data.toBoolean, converter) + case ByteType => wrapperConvertException(data => data.toByte, converter) + case BinaryType => wrapperConvertException(data => data.getBytes, converter) + case IntegerType => wrapperConvertException(data => data.toInt, converter) + case ShortType => wrapperConvertException(data => data.toShort, converter) + case LongType => wrapperConvertException(data => data.toLong, converter) + case FloatType => wrapperConvertException(data => data.toFloat, converter) + case DoubleType => wrapperConvertException(data => data.toDouble, converter) + case _: DecimalType => wrapperConvertException(data => BigDecimal(data), converter) + case DateType if conf.datetimeJava8ApiEnabled => + wrapperConvertException(data => DateTimeUtils.stringToDate( + UTF8String.fromString(data), + DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .map(DateTimeUtils.daysToLocalDate).orNull, converter) + case DateType => wrapperConvertException(data => DateTimeUtils.stringToDate( + UTF8String.fromString(data), + DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .map(DateTimeUtils.toJavaDate).orNull, converter) + case TimestampType if conf.datetimeJava8ApiEnabled => + wrapperConvertException(data => DateTimeUtils.stringToTimestamp( + UTF8String.fromString(data), + DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .map(DateTimeUtils.microsToInstant).orNull, converter) + case TimestampType => wrapperConvertException(data => DateTimeUtils.stringToTimestamp( + UTF8String.fromString(data), + DateTimeUtils.getZoneId(conf.sessionLocalTimeZone)) + .map(DateTimeUtils.toJavaTimestamp).orNull, converter) + case CalendarIntervalType => wrapperConvertException( + data => IntervalUtils.stringToInterval(UTF8String.fromString(data)), + converter) + case udt: UserDefinedType[_] => + wrapperConvertException(data => udt.deserialize(data), converter) + case _ => wrapperConvertException(data => data, converter) + } + } + + // Keep consistent with Hive `LazySimpleSerde`, when there is a type case error, return null + private val wrapperConvertException: (String => Any, Any => Any) => String => Any = + (f: String => Any, converter: Any => Any) => + (data: String) => converter { + try { + f(data) + } catch { + case _: Exception => null Review comment: `NonFatal`? ########## File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql ########## @@ -0,0 +1,72 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES Review comment: nit: `t1` -> `t` ########## File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala ########## @@ -206,75 +169,83 @@ class HiveScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with val query = sql( s""" - |SELECT - |TRANSFORM(a, b, c, d, e) - |USING 'python $scriptFilePath' AS (a, b, c, d, e) - |FROM v + |SELECT TRANSFORM(a, b, c, d, e) + |USING 'python ${scriptFilePath}' + |FROM v """.stripMargin) - // In Hive 1.2, the string representation of a decimal omits trailing zeroes. - // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. - val decimalToString: Column => Column = if (HiveUtils.isHive23) { - c => c.cast("string") - } else { - c => c.cast("decimal(1, 0)").cast("string") - } - checkAnswer(query, identity, df.select( - 'a.cast("string"), - 'b.cast("string"), - 'c.cast("string"), - decimalToString('d), - 'e.cast("string")).collect()) + // In hive default serde mode, if we don't define output schema, it will choose first + // two column as output schema (key: String, value: String) + checkAnswer( + query, + identity, + df.select( + 'a.cast("string").as("key"), + 'b.cast("string").as("value")).collect()) } } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (no serde)") { + test("SPARK-32106: TRANSFORM support complex data types as input and ouput type (hive serde)") { assume(TestUtils.testCommandAvailable("/bin/bash")) + withTempView("v") { + val df = Seq( + (1, "1", Array(0, 1, 2), Map("a" -> 1)), + (2, "2", Array(3, 4, 5), Map("b" -> 2))) + .toDF("a", "b", "c", "d") + .select('a, 'b, 'c, 'd, struct('a, 'b).as("e")) + df.createTempView("v") - val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") - val e = intercept[SparkException] { - val plan = - new HiveScriptTransformationExec( - input = Seq(rowsDf.col("a").expr), - script = "some_non_existent_command", - output = Seq(AttributeReference("a", StringType)()), - child = rowsDf.queryExecution.sparkPlan, - ioschema = noSerdeIOSchema) - SparkPlanTest.executePlan(plan, hiveContext) + // Hive serde support ArrayType/MapType/StructType as input and output data type + checkAnswer( + df, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq( + df.col("c").expr, + df.col("d").expr, + df.col("e").expr), + script = "cat", + output = Seq( + AttributeReference("c", ArrayType(IntegerType))(), + AttributeReference("d", MapType(StringType, IntegerType))(), + AttributeReference("e", StructType( + Seq( + StructField("col1", IntegerType, false), + StructField("col2", StringType, true))))()), + child = child, + ioschema = serdeIOSchema + ), + df.select('c, 'd, 'e).collect()) } - assert(e.getMessage.contains("Subprocess exited with status")) - assert(uncaughtExceptionHandler.exception.isEmpty) } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (with serde)") { + test("SPARK-32106: TRANSFORM don't support CalenderIntervalType/UserDefinedType (hive serde)") { assume(TestUtils.testCommandAvailable("/bin/bash")) + withTempView("v") { + val df = Seq( + (1, new CalendarInterval(7, 1, 1000), new TestUDT.MyDenseVector(Array(1, 2, 3))), + (1, new CalendarInterval(7, 1, 1000), new TestUDT.MyDenseVector(Array(1, 2, 3)))) + .toDF("a", "b", "c") + df.createTempView("v") - val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") - val e = intercept[SparkException] { - val plan = - new HiveScriptTransformationExec( - input = Seq(rowsDf.col("a").expr), - script = "some_non_existent_command", - output = Seq(AttributeReference("a", StringType)()), - child = rowsDf.queryExecution.sparkPlan, - ioschema = serdeIOSchema) - SparkPlanTest.executePlan(plan, hiveContext) + val e1 = intercept[Exception] { + sql( + """ + |SELECT TRANSFORM(a, b) USING 'cat' AS (a, b) + |FROM v + """.stripMargin).collect() + } + assert(e1.getMessage.contains("scala.MatchError: CalendarIntervalType")) Review comment: `MatchError`? Could we make this more suitable one, e.g., SparkException? ########## File path: sql/core/src/test/resources/sql-tests/results/transform.sql.out ########## @@ -0,0 +1,160 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +as t1(a, b, c, d, e, f, g, h, i, j, k, l) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t1 +-- !query schema +struct<a:string> +-- !query output +1 +2 +3 + + +-- !query +SELECT TRANSFORM(a) +USING 'some_non_existent_command' AS (a) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Subprocess exited with status 127. Error: /bin/bash: some_non_existent_command: command not found + + +-- !query +SELECT TRANSFORM(a) +USING 'python some_non_existent_file' AS (a) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Subprocess exited with status 2. Error: python: can't open file 'some_non_existent_file': [Errno 2] No such file or directory + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t1 +) tmp +-- !query schema +struct<a:string,b:boolean,decode(c, UTF-8):string,d:tinyint,e:int,f:smallint,g:bigint,h:float,i:double,j:decimal(38,18),k:timestamp,l:date> +-- !query output +1 true Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 1997-01-02 00:00:00 2000-04-01 +2 false Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 1997-01-02 03:04:05 2000-04-02 +3 true Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 1997-02-10 17:32:01 2000-04-03 + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' +FROM t1 +-- !query schema +struct<> +-- !query output +java.lang.ArrayIndexOutOfBoundsException Review comment: What's the hive behaviour in this case? ########## File path: sql/core/src/test/resources/sql-tests/results/transform.sql.out ########## @@ -0,0 +1,160 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +as t1(a, b, c, d, e, f, g, h, i, j, k, l) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t1 +-- !query schema +struct<a:string> +-- !query output +1 +2 +3 + + +-- !query +SELECT TRANSFORM(a) +USING 'some_non_existent_command' AS (a) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Subprocess exited with status 127. Error: /bin/bash: some_non_existent_command: command not found + + +-- !query +SELECT TRANSFORM(a) +USING 'python some_non_existent_file' AS (a) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +Subprocess exited with status 2. Error: python: can't open file 'some_non_existent_file': [Errno 2] No such file or directory + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t1 +) tmp +-- !query schema +struct<a:string,b:boolean,decode(c, UTF-8):string,d:tinyint,e:int,f:smallint,g:bigint,h:float,i:double,j:decimal(38,18),k:timestamp,l:date> +-- !query output +1 true Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 1997-01-02 00:00:00 2000-04-01 +2 false Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 1997-01-02 03:04:05 2000-04-02 +3 true Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 1997-02-10 17:32:01 2000-04-03 + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' +FROM t1 +-- !query schema +struct<> +-- !query output +java.lang.ArrayIndexOutOfBoundsException +1 + + +-- !query +SELECT TRANSFORM(a, b) +USING 'cat' +FROM t1 +-- !query schema +struct<key:string,value:string> +-- !query output +1 true +2 false +3 true + + +-- !query +SELECT TRANSFORM(a, b, c) Review comment: In this case, hive has different output? ``` hive> create table t (c0 int, c1 int, c2 int); hive> INSERT INTO t VALUES (1, 1, 1); hive> INSERT INTO t VALUES (2, 2, 2); hive> CREATE VIEW v AS SELECT TRANSFORM(c0, c1, c2) USING 'cat' FROM t; hive> DESCRIBE v; key string value string hive> SELECT * FROM v; 1 1 1 2 2 2 hive> SELECT key FROM v; 1 2 hive> SELECT value FROM v; 1 1 2 2 ``` Yea, I know the current behaviour follows the v3.0.0 one though. ########## File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala ########## @@ -206,75 +169,83 @@ class HiveScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with val query = sql( s""" - |SELECT - |TRANSFORM(a, b, c, d, e) - |USING 'python $scriptFilePath' AS (a, b, c, d, e) - |FROM v + |SELECT TRANSFORM(a, b, c, d, e) + |USING 'python ${scriptFilePath}' + |FROM v """.stripMargin) - // In Hive 1.2, the string representation of a decimal omits trailing zeroes. - // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. - val decimalToString: Column => Column = if (HiveUtils.isHive23) { - c => c.cast("string") - } else { - c => c.cast("decimal(1, 0)").cast("string") - } - checkAnswer(query, identity, df.select( - 'a.cast("string"), - 'b.cast("string"), - 'c.cast("string"), - decimalToString('d), - 'e.cast("string")).collect()) + // In hive default serde mode, if we don't define output schema, it will choose first + // two column as output schema (key: String, value: String) + checkAnswer( + query, + identity, + df.select( + 'a.cast("string").as("key"), + 'b.cast("string").as("value")).collect()) } } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (no serde)") { + test("SPARK-32106: TRANSFORM support complex data types as input and ouput type (hive serde)") { assume(TestUtils.testCommandAvailable("/bin/bash")) + withTempView("v") { + val df = Seq( + (1, "1", Array(0, 1, 2), Map("a" -> 1)), + (2, "2", Array(3, 4, 5), Map("b" -> 2))) + .toDF("a", "b", "c", "d") + .select('a, 'b, 'c, 'd, struct('a, 'b).as("e")) + df.createTempView("v") - val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") - val e = intercept[SparkException] { - val plan = - new HiveScriptTransformationExec( - input = Seq(rowsDf.col("a").expr), - script = "some_non_existent_command", - output = Seq(AttributeReference("a", StringType)()), - child = rowsDf.queryExecution.sparkPlan, - ioschema = noSerdeIOSchema) - SparkPlanTest.executePlan(plan, hiveContext) + // Hive serde support ArrayType/MapType/StructType as input and output data type + checkAnswer( + df, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq( + df.col("c").expr, + df.col("d").expr, + df.col("e").expr), + script = "cat", + output = Seq( + AttributeReference("c", ArrayType(IntegerType))(), + AttributeReference("d", MapType(StringType, IntegerType))(), + AttributeReference("e", StructType( + Seq( + StructField("col1", IntegerType, false), + StructField("col2", StringType, true))))()), + child = child, + ioschema = serdeIOSchema + ), + df.select('c, 'd, 'e).collect()) } - assert(e.getMessage.contains("Subprocess exited with status")) - assert(uncaughtExceptionHandler.exception.isEmpty) } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (with serde)") { + test("SPARK-32106: TRANSFORM don't support CalenderIntervalType/UserDefinedType (hive serde)") { Review comment: `don't ` -> `doesn't` ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala ########## @@ -0,0 +1,343 @@ +/* + * 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 + +import java.sql.{Date, Timestamp} + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ +import org.scalatest.Assertions._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.{SparkException, TaskContext, TestUtils} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestUtils + with BeforeAndAfterEach { + import testImplicits._ + import ScriptTransformationIOSchema._ + + protected val uncaughtExceptionHandler = new TestUncaughtExceptionHandler + + private var defaultUncaughtExceptionHandler: Thread.UncaughtExceptionHandler = _ + + protected override def beforeAll(): Unit = { + super.beforeAll() + defaultUncaughtExceptionHandler = Thread.getDefaultUncaughtExceptionHandler + Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler) + } + + protected override def afterAll(): Unit = { + super.afterAll() + Thread.setDefaultUncaughtExceptionHandler(defaultUncaughtExceptionHandler) + } + + override protected def afterEach(): Unit = { + super.afterEach() + uncaughtExceptionHandler.cleanStatus() + } + + def isHive23OrSpark: Boolean + + def createScriptTransformationExec( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: SparkPlan, + ioschema: ScriptTransformationIOSchema): BaseScriptTransformationExec + + test("cat without SerDe") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") + checkAnswer( + rowsDf, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq(rowsDf.col("a").expr), + script = "cat", + output = Seq(AttributeReference("a", StringType)()), + child = child, + ioschema = defaultIOSchema + ), + rowsDf.collect()) + assert(uncaughtExceptionHandler.exception.isEmpty) + } + + test("script transformation should not swallow errors from upstream operators (no serde)") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") + val e = intercept[TestFailedException] { + checkAnswer( + rowsDf, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq(rowsDf.col("a").expr), + script = "cat", + output = Seq(AttributeReference("a", StringType)()), + child = ExceptionInjectingOperator(child), + ioschema = defaultIOSchema + ), + rowsDf.collect()) + } + assert(e.getMessage().contains("intentional exception")) + // Before SPARK-25158, uncaughtExceptionHandler will catch IllegalArgumentException + assert(uncaughtExceptionHandler.exception.isEmpty) + } + + test("SPARK-25990: TRANSFORM should handle different data types correctly") { + assume(TestUtils.testCommandAvailable("python")) + val scriptFilePath = getTestResourcePath("test_script.py") + + withTempView("v") { + val df = Seq( + (1, "1", 1.0, BigDecimal(1.0), new Timestamp(1)), + (2, "2", 2.0, BigDecimal(2.0), new Timestamp(2)), + (3, "3", 3.0, BigDecimal(3.0), new Timestamp(3)) + ).toDF("a", "b", "c", "d", "e") // Note column d's data type is Decimal(38, 18) + df.createTempView("v") + + val query = sql( + s""" + |SELECT + |TRANSFORM(a, b, c, d, e) + |USING 'python $scriptFilePath' AS (a, b, c, d, e) + |FROM v + """.stripMargin) + + // In Hive 1.2, the string representation of a decimal omits trailing zeroes. + // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. + val decimalToString: Column => Column = if (isHive23OrSpark) { + c => c.cast("string") + } else { + c => c.cast("decimal(1, 0)").cast("string") + } + checkAnswer(query, identity, df.select( + 'a.cast("string"), + 'b.cast("string"), + 'c.cast("string"), + decimalToString('d), + 'e.cast("string")).collect()) + } + } + + test("SPARK-25990: TRANSFORM should handle schema less correctly (no serde)") { + assume(TestUtils.testCommandAvailable("python")) + val scriptFilePath = getTestResourcePath("test_script.py") + + withTempView("v") { + val df = Seq( + (1, "1", 1.0, BigDecimal(1.0), new Timestamp(1)), + (2, "2", 2.0, BigDecimal(2.0), new Timestamp(2)), + (3, "3", 3.0, BigDecimal(3.0), new Timestamp(3)) + ).toDF("a", "b", "c", "d", "e") // Note column d's data type is Decimal(38, 18) + + checkAnswer( + df, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq( + df.col("a").expr, + df.col("b").expr, + df.col("c").expr, + df.col("d").expr, + df.col("e").expr), + script = s"python $scriptFilePath", + output = Seq( + AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), + child = child, + ioschema = defaultIOSchema.copy(schemaLess = true) + ), + df.select( + 'a.cast("string").as("key"), + 'b.cast("string").as("value")).collect()) + } + } + + test("SPARK-30973: TRANSFORM should wait for the termination of the script (no serde)") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") + val e = intercept[SparkException] { + val plan = + createScriptTransformationExec( + input = Seq(rowsDf.col("a").expr), + script = "some_non_existent_command", + output = Seq(AttributeReference("a", StringType)()), + child = rowsDf.queryExecution.sparkPlan, + ioschema = defaultIOSchema) + SparkPlanTest.executePlan(plan, spark.sqlContext) + } + assert(e.getMessage.contains("Subprocess exited with status")) + assert(uncaughtExceptionHandler.exception.isEmpty) + } + + test("SPARK-32106: TRANSFORM should support all data types as input (no serde)") { + assume(TestUtils.testCommandAvailable("python")) + Array(false, true).foreach { java8AapiEnable => + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8AapiEnable.toString) { + withTempView("v") { + val df = Seq( + (1, "1", 1.0, 11.toByte, BigDecimal(1.0), new Timestamp(1), Review comment: `float` is missing? ########## File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala ########## @@ -206,75 +169,83 @@ class HiveScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with val query = sql( s""" - |SELECT - |TRANSFORM(a, b, c, d, e) - |USING 'python $scriptFilePath' AS (a, b, c, d, e) - |FROM v + |SELECT TRANSFORM(a, b, c, d, e) + |USING 'python ${scriptFilePath}' + |FROM v """.stripMargin) - // In Hive 1.2, the string representation of a decimal omits trailing zeroes. - // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. - val decimalToString: Column => Column = if (HiveUtils.isHive23) { - c => c.cast("string") - } else { - c => c.cast("decimal(1, 0)").cast("string") - } - checkAnswer(query, identity, df.select( - 'a.cast("string"), - 'b.cast("string"), - 'c.cast("string"), - decimalToString('d), - 'e.cast("string")).collect()) + // In hive default serde mode, if we don't define output schema, it will choose first + // two column as output schema (key: String, value: String) + checkAnswer( + query, + identity, + df.select( + 'a.cast("string").as("key"), + 'b.cast("string").as("value")).collect()) } } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (no serde)") { + test("SPARK-32106: TRANSFORM support complex data types as input and ouput type (hive serde)") { assume(TestUtils.testCommandAvailable("/bin/bash")) + withTempView("v") { + val df = Seq( + (1, "1", Array(0, 1, 2), Map("a" -> 1)), + (2, "2", Array(3, 4, 5), Map("b" -> 2))) + .toDF("a", "b", "c", "d") + .select('a, 'b, 'c, 'd, struct('a, 'b).as("e")) + df.createTempView("v") - val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") - val e = intercept[SparkException] { - val plan = - new HiveScriptTransformationExec( - input = Seq(rowsDf.col("a").expr), - script = "some_non_existent_command", - output = Seq(AttributeReference("a", StringType)()), - child = rowsDf.queryExecution.sparkPlan, - ioschema = noSerdeIOSchema) - SparkPlanTest.executePlan(plan, hiveContext) + // Hive serde support ArrayType/MapType/StructType as input and output data type + checkAnswer( + df, + (child: SparkPlan) => createScriptTransformationExec( + input = Seq( + df.col("c").expr, + df.col("d").expr, + df.col("e").expr), + script = "cat", + output = Seq( + AttributeReference("c", ArrayType(IntegerType))(), + AttributeReference("d", MapType(StringType, IntegerType))(), + AttributeReference("e", StructType( + Seq( + StructField("col1", IntegerType, false), + StructField("col2", StringType, true))))()), + child = child, + ioschema = serdeIOSchema + ), + df.select('c, 'd, 'e).collect()) } - assert(e.getMessage.contains("Subprocess exited with status")) - assert(uncaughtExceptionHandler.exception.isEmpty) } - test("SPARK-30973: TRANSFORM should wait for the termination of the script (with serde)") { + test("SPARK-32106: TRANSFORM don't support CalenderIntervalType/UserDefinedType (hive serde)") { assume(TestUtils.testCommandAvailable("/bin/bash")) + withTempView("v") { + val df = Seq( + (1, new CalendarInterval(7, 1, 1000), new TestUDT.MyDenseVector(Array(1, 2, 3))), + (1, new CalendarInterval(7, 1, 1000), new TestUDT.MyDenseVector(Array(1, 2, 3)))) + .toDF("a", "b", "c") + df.createTempView("v") - val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") - val e = intercept[SparkException] { - val plan = - new HiveScriptTransformationExec( - input = Seq(rowsDf.col("a").expr), - script = "some_non_existent_command", - output = Seq(AttributeReference("a", StringType)()), - child = rowsDf.queryExecution.sparkPlan, - ioschema = serdeIOSchema) - SparkPlanTest.executePlan(plan, hiveContext) + val e1 = intercept[Exception] { + sql( + """ + |SELECT TRANSFORM(a, b) USING 'cat' AS (a, b) + |FROM v + """.stripMargin).collect() + } + assert(e1.getMessage.contains("scala.MatchError: CalendarIntervalType")) + + val e2 = intercept[Exception] { + sql( + """ + |SELECT TRANSFORM(a, c) USING 'cat' AS (a, c) + |FROM v + """.stripMargin).collect() + } + assert(e2.getMessage.contains( + "scala.MatchError: org.apache.spark.sql.types.TestUDT$MyDenseVectorUDT")) Review comment: ditto ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org