-
Notifications
You must be signed in to change notification settings - Fork 28.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-32106][SQL] Implement script transform in sql/core
### What changes were proposed in this pull request? * Implement `SparkScriptTransformationExec` based on `BaseScriptTransformationExec` * Implement `SparkScriptTransformationWriterThread` based on `BaseScriptTransformationWriterThread` of writing data * Add rule `SparkScripts` to support convert script LogicalPlan to SparkPlan in Spark SQL (without hive mode) * Add `SparkScriptTransformationSuite` test spark spec case * add test in `SQLQueryTestSuite` And we will close #29085 . ### Why are the changes needed? Support user use Script Transform without Hive ### Does this PR introduce _any_ user-facing change? User can use Script Transformation without hive in no serde mode. Such as : **default no serde ** ``` SELECT TRANSFORM(a, b, c) USING 'cat' AS (a int, b string, c long) FROM testData ``` **no serde with spec ROW FORMAT DELIMITED** ``` SELECT TRANSFORM(a, b, c) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY '\u0002' MAP KEYS TERMINATED BY '\u0003' LINES TERMINATED BY '\n' NULL DEFINED AS 'null' USING 'cat' AS (a, b, c) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY '\u0004' MAP KEYS TERMINATED BY '\u0005' LINES TERMINATED BY '\n' NULL DEFINED AS 'NULL' FROM testData ``` ### How was this patch tested? Added UT Closes #29414 from AngersZhuuuu/SPARK-32106-MINOR. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
- Loading branch information
1 parent
f62e957
commit 7466031
Showing
11 changed files
with
982 additions
and
65 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkScriptTransformationExec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* 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.io._ | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
|
||
import org.apache.spark.TaskContext | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.util.CircularBuffer | ||
|
||
/** | ||
* Transforms the input by forking and running the specified script. | ||
* | ||
* @param input the set of expression that should be passed to the script. | ||
* @param script the command that should be executed. | ||
* @param output the attributes that are produced by the script. | ||
* @param child logical plan whose output is transformed. | ||
* @param ioschema the class set that defines how to handle input/output data. | ||
*/ | ||
case class SparkScriptTransformationExec( | ||
input: Seq[Expression], | ||
script: String, | ||
output: Seq[Attribute], | ||
child: SparkPlan, | ||
ioschema: ScriptTransformationIOSchema) | ||
extends BaseScriptTransformationExec { | ||
|
||
override def processIterator( | ||
inputIterator: Iterator[InternalRow], | ||
hadoopConf: Configuration): Iterator[InternalRow] = { | ||
|
||
val (outputStream, proc, inputStream, stderrBuffer) = initProc | ||
|
||
val outputProjection = new InterpretedProjection(inputExpressionsWithoutSerde, child.output) | ||
|
||
// This new thread will consume the ScriptTransformation's input rows and write them to the | ||
// external process. That process's output will be read by this current thread. | ||
val writerThread = SparkScriptTransformationWriterThread( | ||
inputIterator.map(outputProjection), | ||
inputExpressionsWithoutSerde.map(_.dataType), | ||
ioschema, | ||
outputStream, | ||
proc, | ||
stderrBuffer, | ||
TaskContext.get(), | ||
hadoopConf | ||
) | ||
|
||
val outputIterator = | ||
createOutputIteratorWithoutSerde(writerThread, inputStream, proc, stderrBuffer) | ||
|
||
writerThread.start() | ||
|
||
outputIterator | ||
} | ||
} | ||
|
||
case class SparkScriptTransformationWriterThread( | ||
iter: Iterator[InternalRow], | ||
inputSchema: Seq[DataType], | ||
ioSchema: ScriptTransformationIOSchema, | ||
outputStream: OutputStream, | ||
proc: Process, | ||
stderrBuffer: CircularBuffer, | ||
taskContext: TaskContext, | ||
conf: Configuration) | ||
extends BaseScriptTransformationWriterThread { | ||
|
||
override def processRows(): Unit = { | ||
processRowsWithoutSerde() | ||
} | ||
} |
Oops, something went wrong.