maropu commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r458498021



##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), float(3.0), 3.0, Decimal(3.0), 
timestamp('1997-02-10 17:32:01-08'), date('2000-04-03'))

Review comment:
       Where's `int`? And, we need `array_position(array(3, 2, 1), 1)` for 
tests? `array_position(array(3, 2, 1), 1)` -> `bigint(3)`?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
##########
@@ -258,6 +258,7 @@ class SQLQueryTestSuite extends QueryTest with 
SharedSparkSession with SQLHelper
       newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER")
     }
 
+    assume(TestUtils.testCommandAvailable("/bin/bash"))

Review comment:
       Please add some comments about why this needed here.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##########
@@ -87,17 +175,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 =>

Review comment:
       Could you add tests in `BaseScriptTransformationSuite.scala ` with this 
flag enabled/disabled?

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+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;
+
+
+-- support different data type
+SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k FROM (
+    SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k)
+    USING 'cat' AS (
+        a string,
+        b boolean,
+        c binary,
+        d tinyint,
+        e smallint,
+        f long,
+        g float,
+        h double,
+        i decimal(38, 18),
+        j timestamp,
+        k date)
+    FROM t1
+) tmp;
+
+
+-- handle schema less
+SELECT TRANSFORM(a, b)
+USING 'cat'
+FROM t1;
+
+-- return null when return string incompatible (no serde)
+SELECT TRANSFORM(a, b, c)
+USING 'cat' as (a int, b int , c int)

Review comment:
       nit: remove  a space: `int ,`

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
##########
@@ -0,0 +1,367 @@
+/*
+ * 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, Row}
+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.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)
+
+      // 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(
+        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"),
+          concat_ws("\t",
+            'b.cast("string"),
+            'c.cast("string"),
+            decimalToString('d),
+            'e.cast("string"))).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"))
+    withTempView("v") {
+      val df = Seq(
+        (1, "1", 1.0, 11.toByte, BigDecimal(1.0), new Timestamp(1),
+          new Date(2020, 7, 1), new CalendarInterval(7, 1, 1000), Array(0, 1, 
2),
+          Map("a" -> 1), new TestUDT.MyDenseVector(Array(1, 2, 3)), new 
SimpleTuple(1, 1L)),
+        (2, "2", 2.0, 22.toByte, BigDecimal(2.0), new Timestamp(2),
+          new Date(2020, 7, 2), new CalendarInterval(7, 2, 2000), Array(3, 4, 
5),
+          Map("b" -> 2), new TestUDT.MyDenseVector(Array(1, 2, 3)), new 
SimpleTuple(1, 1L)),
+        (3, "3", 3.0, 33.toByte, BigDecimal(3.0), new Timestamp(3),
+          new Date(2020, 7, 3), new CalendarInterval(7, 3, 3000), Array(6, 7, 
8),
+          Map("c" -> 3), new TestUDT.MyDenseVector(Array(1, 2, 3)), new 
SimpleTuple(1, 1L))
+      ).toDF("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l")
+        .select('a, 'b, 'c, 'd, 'e, 'f, 'g, 'h, 'i, 'j, 'k, 'l,
+          struct('a, 'b).as("m"), unhex('a).as("n"), lit(true).as("o")
+        ) // Note column d's data type is Decimal(38, 18)
+
+      // Can't support convert script output data to 
ArrayType/MapType/StructType now,
+      // return these column still as string.
+      // For UserDefinedType, if user defined deserialize method to support 
convert string
+      // to UserType like [[SimpleTupleUDT]], we can support convert to this 
UDT, else we
+      // will return null value as column.
+      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,
+            df.col("f").expr,
+            df.col("g").expr,
+            df.col("h").expr,
+            df.col("i").expr,
+            df.col("j").expr,
+            df.col("k").expr,
+            df.col("l").expr,
+            df.col("m").expr,
+            df.col("n").expr,
+            df.col("o").expr),
+          script = "cat",
+          output = Seq(
+            AttributeReference("a", IntegerType)(),
+            AttributeReference("b", StringType)(),
+            AttributeReference("c", DoubleType)(),
+            AttributeReference("d", ByteType)(),
+            AttributeReference("e", DecimalType(38, 18))(),
+            AttributeReference("f", TimestampType)(),
+            AttributeReference("g", DateType)(),
+            AttributeReference("h", CalendarIntervalType)(),
+            AttributeReference("i", StringType)(),
+            AttributeReference("j", StringType)(),
+            AttributeReference("k", StringType)(),
+            AttributeReference("l", new SimpleTupleUDT)(),
+            AttributeReference("m", StringType)(),
+            AttributeReference("n", BinaryType)(),
+            AttributeReference("o", BooleanType)()),
+          child = child,
+          ioschema = defaultIOSchema
+        ),
+        df.select(
+          'a, 'b, 'c, 'd, 'e, 'f, 'g, 'h,
+          'i.cast("string"),
+          'j.cast("string"),
+          'k.cast("string"),
+          'l, 'm.cast("string"), 'n, 'o).collect())
+    }
+  }
+
+  test("SPARK-32106: TRANSFORM should return null when return string 
incompatible") {
+    checkAnswer(
+      sql(
+        """
+          |SELECT TRANSFORM(a, b, c)
+          |USING 'cat' as (a int, b int , c int)
+          |FROM (
+          |SELECT
+          |1 AS a,
+          |"a" AS b,
+          |CAST(2000 AS timestamp) AS c
+          |) tmp
+        """.stripMargin),
+      identity,
+      Row(1, null, null) :: Nil)
+  }

Review comment:
       This part is duplicated with `SQLQueryTestSuite`, so we can remove it. 
Instead, I think it would better to add more tests for all the incompatible 
patterns (`string`->short, `string`->long, ...) for better test coverage. Could 
you?

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+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;
+
+
+-- support different data type
+SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k FROM (
+    SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k)
+    USING 'cat' AS (
+        a string,
+        b boolean,
+        c binary,
+        d tinyint,
+        e smallint,
+        f long,
+        g float,
+        h double,
+        i decimal(38, 18),
+        j timestamp,
+        k date)
+    FROM t1
+) tmp;
+
+
+-- handle schema less
+SELECT TRANSFORM(a, b)
+USING 'cat'
+FROM t1;
+
+-- return null when return string incompatible (no serde)
+SELECT TRANSFORM(a, b, c)
+USING 'cat' as (a int, b int , c int)
+FROM (
+    SELECT

Review comment:
       We need the subquery for this test?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##########
@@ -87,17 +175,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 =>

Review comment:
       ditto

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##########
@@ -56,10 +69,85 @@ 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 {
+        prevLine: String =>
+          new GenericInternalRow(
+            prevLine.split(outputRowFormat, 2)
+              .map(CatalystTypeConverters.convertToCatalyst))

Review comment:
       In this case, we always handle output as strings? If so, how about using 
`CatalystTypeConverters.createToCatalystConverter(string type)` instead of 
`CatalystTypeConverters.convertToCatalyst`?

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+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;
+
+
+-- support different data type
+SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k FROM (
+    SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k)
+    USING 'cat' AS (
+        a string,
+        b boolean,
+        c binary,
+        d tinyint,
+        e smallint,
+        f long,
+        g float,
+        h double,
+        i decimal(38, 18),
+        j timestamp,
+        k date)
+    FROM t1
+) tmp;
+
+
+-- handle schema less
+SELECT TRANSFORM(a, b)

Review comment:
       How about the test cases, `TRANSFORM(a)` and `TRANSFORM(a, b, c...)` 
with schemaless enabled?

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+SELECT TRANSFORM(a)
+USING 'cat' AS (a)
+FROM t1;
+
+

Review comment:
       nit: remove unnecessary spaces.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##########
@@ -56,10 +69,85 @@ 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 {
+        prevLine: String =>
+          new GenericInternalRow(
+            prevLine.split(outputRowFormat, 2)
+              .map(CatalystTypeConverters.convertToCatalyst))

Review comment:
       Probably, we need to add some comments about how to handle the 
schemaless mode here.

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+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;
+
+
+-- support different data type
+SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k FROM (
+    SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k)
+    USING 'cat' AS (
+        a string,
+        b boolean,
+        c binary,
+        d tinyint,
+        e smallint,
+        f long,
+        g float,
+        h double,
+        i decimal(38, 18),
+        j timestamp,
+        k date)
+    FROM t1
+) tmp;
+
+
+-- handle schema less
+SELECT TRANSFORM(a, b)
+USING 'cat'
+FROM t1;
+
+-- return null when return string incompatible (no serde)
+SELECT TRANSFORM(a, b, c)
+USING 'cat' as (a int, b int , c int)
+FROM (
+    SELECT
+    1 AS a,
+    "a" AS b,
+    CAST(2000 AS timestamp) AS c
+) tmp;
+
+
+-- transform can't run with aggregation

Review comment:
       How about adding the JIRA ID, SPARK-28227?

##########
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), smallint(100), 
array_position(array(3, 2, 1), 1), float(1.0), 1.0, Decimal(1.0), 
timestamp('1997-01-02'), date('2000-04-01')),
+('2', false, unhex('537061726B2053514C'), tinyint(2), smallint(200), 
array_position(array(3, 2, 1), 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), smallint(300), 
array_position(array(3, 2, 1), 1), 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);
+
+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;
+
+
+-- support different data type

Review comment:
       What does `different` means here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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

Reply via email to