-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-32106][SQL] Implement script transform in sql/core #29414
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
b4c4b40
[SPARK-32106][SQL] Implement script transform in sql/core
AngersZhuuuu c58729b
Update SparkSqlParser.scala
AngersZhuuuu fc5bbce
follow comment
AngersZhuuuu a812333
follow commnet
AngersZhuuuu fdfc987
follow comment
AngersZhuuuu 5b17cd9
Merge branch 'master' into SPARK-32106-MINOR
AngersZhuuuu dabae9b
fix add UT
AngersZhuuuu 61466f6
Merge branch 'master' into SPARK-32106-MINOR
AngersZhuuuu d68066e
Update SparkScriptTransformationSuite.scala
AngersZhuuuu 57735bf
Merge branch 'master' into SPARK-32106-MINOR
AngersZhuuuu 3ad44e3
Update SparkSqlParser.scala
AngersZhuuuu 5b41fbc
Update transform.sql.out
AngersZhuuuu 63bdb53
Merge branch 'master' into SPARK-32106-MINOR
AngersZhuuuu d3c65d4
Update SparkSqlParser.scala
AngersZhuuuu 4e66417
Merge branch 'master' into SPARK-32106-MINOR
AngersZhuuuu a10c5a6
Update transform.sql.out
AngersZhuuuu File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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) | ||
AngersZhuuuu marked this conversation as resolved.
Show resolved
Hide resolved
|
||
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.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.