Github user tejasapatil commented on a diff in the pull request: https://github.com/apache/spark/pull/14702#discussion_r78115155 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/script/ScriptTransformationExec.scala --- @@ -0,0 +1,313 @@ +/* + * 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.script + +import java.io._ +import java.nio.charset.StandardCharsets + +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.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils} + +/** + * 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. + */ +private[sql] +case class ScriptTransformationExec( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: SparkPlan, + ioschema: ScriptTransformIOSchema) + extends UnaryExecNode with ScriptTransformBase { + + override def producedAttributes: AttributeSet = outputSet -- inputSet + + protected override def doExecute(): RDD[InternalRow] = + execute(sqlContext, child, schema) + + override def processIterator( + inputIterator: Iterator[InternalRow], + hadoopConf: Configuration) : Iterator[InternalRow] = { + + val (proc, inputStream, outputStream, stderrBuffer, outputProjection) = + init(input, script, child) + + // 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 = new ScriptTransformationWriterThread( + inputIterator, + input.map(_.dataType), + outputProjection, + ioschema, + outputStream, + proc, + stderrBuffer, + TaskContext.get(), + hadoopConf + ) + + val reader = createReader(inputStream) + + val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] { + var curLine: String = null + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + + override def hasNext: Boolean = { + try { + if (curLine == null) { + curLine = reader.readLine() + if (curLine == null) { + checkFailureAndPropagate(writerThread.exception, null, proc, stderrBuffer) + return false + } + } + true + } catch { + case NonFatal(e) => + // If this exception is due to abrupt / unclean termination of `proc`, + // then detect it and propagate a better exception message for end users + checkFailureAndPropagate(writerThread.exception, e, proc, stderrBuffer) + + throw e + } + } + + override def next(): InternalRow = { + if (!hasNext) { + throw new NoSuchElementException + } + val prevLine = curLine + curLine = reader.readLine() + if (!ioschema.isSchemaLess) { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + .map(CatalystTypeConverters.convertToCatalyst)) + } else { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .map(CatalystTypeConverters.convertToCatalyst)) + } + } + } + + writerThread.start() + outputIterator + } +} + +private[sql] trait ScriptTransformBase extends Serializable with Logging { + + def init(input: Seq[Expression], --- End diff -- Fixed the ident. Is there any wiki which has instructions on the code style to be used ? For every PR in past I have manually adjusted the formatting as per rest code.. its about time to ask.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org