AngersZhuuuu commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r456751726
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##########
@@ -56,10 +65,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 processRowWithoutSerde = if (!ioschema.schemaLess) {
+ prevLine: String =>
+ new GenericInternalRow(
+
prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"))
+ .zip(fieldWriters)
+ .map { case (data, writer) => writer(data) })
+ } else {
+ prevLine: String =>
+ new GenericInternalRow(
+
prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2)
+ .map(CatalystTypeConverters.convertToCatalyst))
+ }
+
Review comment:
@maropu Here I change for support schema less mode.
In test case I choose not to use sql since hive serde can't support
schemaless mode well in spark's way.
```
[info] - SPARK-25990: TRANSFORM should handle schema less correctly ***
FAILED *** (360 milliseconds)
[info] Results do not match for Spark plan:
[info] HiveScriptTransformation [a#86, b#87, c#88, d#89, e#90], python
/Users/angerszhu/Documents/project/AngersZhu/spark/sql/core/target/test-classes/test_script.py,
[key#96, value#97],
ScriptTransformationIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,
)),List((field.delim,
)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),true)
[info] +- Project [_1#75 AS a#86, _2#76 AS b#87, _3#77 AS c#88, _4#78 AS
d#89, _5#79 AS e#90]
[info] +- LocalTableScan [_1#75, _2#76, _3#77, _4#78, _5#79]
[info]
[info]
[info] == Results ==
[info] !== Expected Answer - 3 == ==
Actual Answer - 3 ==
[info] ![1,1 1.0 1.000000000000000000 1969-12-31
16:00:00.001] [1,1]
[info] ![2,2 2.0 2.000000000000000000 1969-12-31
16:00:00.002] [2,2]
[info] ![3,3 3.0 3.000000000000000000 1969-12-31
16:00:00.003] [3,3] (SparkPlanTest.scala:96)
[info] org.scalatest.exceptions.TestFailedException:
[info] at
org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
[info] at
org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
[info] at
org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
[i
```
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]