Skip to content
Closed
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2918,6 +2918,14 @@ object SQLConf {
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)

val SCRIPT_TRANSFORMATION_COMMAND_WRAPPER =
buildConf("spark.sql.scriptTransformation.commandWrapper")
.internal()
.doc("Command wrapper for executor to execute transformation script.")
.version("3.2.0")
.stringConf
.createWithDefault("")

val SCRIPT_TRANSFORMATION_EXIT_TIMEOUT =
buildConf("spark.sql.scriptTransformation.exitTimeoutInSeconds")
.internal()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,17 @@

package org.apache.spark.sql.execution

import java.io.{BufferedReader, InputStream, InputStreamReader, OutputStream}
import java.io._
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AngersZhuuuu . nit. Please enumerate all next time.

import java.nio.charset.StandardCharsets
import java.util.ArrayList
import java.util.concurrent.TimeUnit

import scala.collection.JavaConverters._
import scala.util.control.NonFatal

import org.apache.hadoop.conf.Configuration

import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.{SparkException, SparkFiles, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
Expand All @@ -46,6 +47,8 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
def child: SparkPlan
def ioschema: ScriptTransformationIOSchema

type ProcParameters = (OutputStream, Process, InputStream, CircularBuffer)

protected lazy val inputExpressionsWithoutSerde: Seq[Expression] = {
input.map(Cast(_, StringType).withTimeZone(conf.sessionLocalTimeZone))
}
Expand All @@ -55,8 +58,11 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
override def outputPartitioning: Partitioning = child.outputPartitioning

override def doExecute(): RDD[InternalRow] = {
val hadoopConf = sqlContext.sessionState.newHadoopConf()
hadoopConf.set(SQLConf.SCRIPT_TRANSFORMATION_COMMAND_WRAPPER.key,
conf.getConf(SQLConf.SCRIPT_TRANSFORMATION_COMMAND_WRAPPER))
val broadcastedHadoopConf =
new SerializableConfiguration(sqlContext.sessionState.newHadoopConf())
new SerializableConfiguration(hadoopConf)

child.execute().mapPartitions { iter =>
if (iter.hasNext) {
Expand All @@ -69,9 +75,19 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
}
}

protected def initProc: (OutputStream, Process, InputStream, CircularBuffer) = {
val cmd = List("/bin/bash", "-c", script)
protected def initProc(hadoopConf: Configuration): ProcParameters = {
val wrapper = splitArgs(hadoopConf.get(SQLConf.SCRIPT_TRANSFORMATION_COMMAND_WRAPPER.key))
val cmdArgs = splitArgs(script)
val prog = cmdArgs(0)
if(!new File(prog).isAbsolute) {
val progFile = new File(SparkFiles.get(prog))
if (progFile.exists()) {
cmdArgs(0) = progFile.getAbsolutePath
}
}
val cmd = wrapper.toList ++ cmdArgs.toList
val builder = new ProcessBuilder(cmd.asJava)
.directory(new File(SparkFiles.getRootDirectory()))

val proc = builder.start()
val inputStream = proc.getInputStream
Expand Down Expand Up @@ -181,6 +197,55 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
}
}

def splitArgs(args: String): Array[String] = {
val OUTSIDE = 1
val SINGLEQ = 2
val DOUBLEQ = 3
val argList = new ArrayList[String]
val ch = args.toCharArray
val clen = ch.length
var state = OUTSIDE
var argstart = 0
var c = 0
while (c <= clen) {
val last = c == clen
var lastState = state
var endToken = false
if (!last) {
if (ch(c) == '\'') {
if (state == OUTSIDE) {
state = SINGLEQ
} else if (state == SINGLEQ) {
state = OUTSIDE
}
endToken = state != lastState
} else if (ch(c) == '"') {
if (state == OUTSIDE) {
state = DOUBLEQ
} else if (state == DOUBLEQ) {
state = OUTSIDE
}
endToken = state != lastState
} else if (ch(c) == ' ') {
if (state == OUTSIDE) {
endToken = true
}
}
}
if (last || endToken) {
if (c == argstart) {
// unquoted space
} else {
argList.add(args.substring(argstart, c))
}
argstart = c + 1
lastState = state
}
c += 1
}
argList.toArray(new Array[String](0))
}

private lazy val outputFieldWriters: Seq[String => Any] = output.map { attr =>
val converter = CatalystTypeConverters.createToCatalystConverter(attr.dataType)
attr.dataType match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ case class SparkScriptTransformationExec(
inputIterator: Iterator[InternalRow],
hadoopConf: Configuration): Iterator[InternalRow] = {

val (outputStream, proc, inputStream, stderrBuffer) = initProc
val (outputStream, proc, inputStream, stderrBuffer) = initProc(hadoopConf)

val outputProjection = new InterpretedProjection(inputExpressionsWithoutSerde, child.output)

Expand Down
2 changes: 2 additions & 0 deletions sql/core/src/test/resources/test_script.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
#! /usr/bin/python
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe, /usr/bin/python3?


# 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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU
ioschema = defaultIOSchema)
SparkPlanTest.executePlan(plan, spark.sqlContext)
}
assert(e.getMessage.contains("Subprocess exited with status"))
assert(e.getMessage.contains("No such file or directory"))
assert(uncaughtExceptionHandler.exception.isEmpty)
}

Expand Down Expand Up @@ -470,6 +470,119 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU
Row("3\u00014\u00015") :: Nil)
}
}

test("SPARK-33934: Check default execute command wrapper") {
assume(TestUtils.testCommandAvailable("python"))
val scriptFilePath = copyAndGetResourceFile("test_script.py", ".py").getAbsoluteFile
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")

// test 'python /path/to/script.py' with local file
checkAnswer(
sql(
s"""
|SELECT
|TRANSFORM(a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
| USING 'python $scriptFilePath' AS (a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
|FROM v
""".stripMargin), identity, df.select(
'a.cast("string"),
'b.cast("string"),
'c.cast("string"),
'd.cast("string"),
'e.cast("string")).collect())

// test '/path/to/script.py' with script not executable
val e1 = intercept[TestFailedException] {
checkAnswer(
sql(
s"""
|SELECT
|TRANSFORM(a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
| USING '$scriptFilePath' AS (a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
|FROM v
""".stripMargin), identity, df.select(
'a.cast("string"),
'b.cast("string"),
'c.cast("string"),
'd.cast("string"),
'e.cast("string")).collect())
}.getMessage
assert(e1.contains("Permission denied"))

// test `/path/to/script.py' with script executable
scriptFilePath.setExecutable(true)
checkAnswer(
sql(
s"""
|SELECT
|TRANSFORM(a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
| USING '$scriptFilePath' AS (a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
|FROM v
""".stripMargin), identity, df.select(
'a.cast("string"),
'b.cast("string"),
'c.cast("string"),
'd.cast("string"),
'e.cast("string")).collect())

scriptFilePath.setExecutable(false)
sql(s"ADD FILE ${scriptFilePath.getAbsolutePath}")

// test `script.py` when file added
checkAnswer(
sql(
s"""
|SELECT TRANSFORM(a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
| USING '${scriptFilePath.getName}' AS (a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
|FROM v
""".stripMargin), identity, df.select(
'a.cast("string"),
'b.cast("string"),
'c.cast("string"),
'd.cast("string"),
'e.cast("string")).collect())

// test `python script.py` when file added
checkAnswer(
sql(
s"""
|SELECT TRANSFORM(a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
| USING 'python ${scriptFilePath.getName}' AS (a, b, c, d, e)
| ROW FORMAT DELIMITED
| FIELDS TERMINATED BY '\t'
|FROM v
""".stripMargin), identity, df.select(
'a.cast("string"),
'b.cast("string"),
'c.cast("string"),
'd.cast("string"),
'e.cast("string")).collect())
}
}
}

case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ case class HiveScriptTransformationExec(
inputIterator: Iterator[InternalRow],
hadoopConf: Configuration): Iterator[InternalRow] = {

val (outputStream, proc, inputStream, stderrBuffer) = initProc
val (outputStream, proc, inputStream, stderrBuffer) = initProc(hadoopConf)

val (inputSerde, inputSoi) = initInputSerDe(ioschema, input).getOrElse((null, null))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T
ioschema = hiveIOSchema)
SparkPlanTest.executePlan(plan, hiveContext)
}
assert(e.getMessage.contains("Subprocess exited with status"))
assert(e.getMessage.contains("No such file or directory"))
assert(uncaughtExceptionHandler.exception.isEmpty)
}

Expand Down Expand Up @@ -150,7 +150,7 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T
ioschema = hiveIOSchema)
SparkPlanTest.executePlan(plan, hiveContext)
}
assert(e.getMessage.contains("Subprocess exited with status"))
assert(e.getMessage.contains("No such file or directory"))
assert(uncaughtExceptionHandler.exception.isEmpty)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1032,7 +1032,12 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
val data = (1 to 100000).map { i => (i, i, i) }
data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans")
assert(0 ===
sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans").count())
sql(
"""
|SELECT TRANSFORM (d1, d2, d3)
| USING '/bin/bash -c \'cat 1>&2\'' AS (a,b,c)
|FROM script_trans
""".stripMargin).count())
}
}

Expand Down