You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/09 15:58:36 UTC

[GitHub] [spark] maropu commented on a change in pull request #27983: [SPARK-31936][SQL]Implement ScriptTransformation in sql/core

maropu commented on a change in pull request #27983:
URL: https://github.com/apache/spark/pull/27983#discussion_r437476009



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/script/ScriptTransformationExec.scala
##########
@@ -0,0 +1,239 @@
+/*
+ * 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 java.sql.Date
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.TaskContext
+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.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.types.{DataType, DateType, TimestampType}
+import org.apache.spark.util.{CircularBuffer, RedirectThread}
+
+/**
+ * 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.
+ */
+case class ScriptTransformationExec(
+    input: Seq[Expression],
+    script: String,
+    output: Seq[Attribute],
+    child: SparkPlan,
+    ioschema: ScriptTransformIOSchema)
+  extends ScriptTransformBase {
+
+  override def producedAttributes: AttributeSet = outputSet -- inputSet
+
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+
+  override def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration)
+  : Iterator[InternalRow] = {
+    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,
+      "Thread-ScriptTransformation-STDERR-Consumer").start()
+
+    val outputProjection = new InterpretedProjection(input, 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 = new ScriptTransformationWriterThread(
+      inputIterator.map(outputProjection),
+      input.map(_.dataType),
+      ioschema,
+      outputStream,
+      proc,
+      stderrBuffer,
+      TaskContext.get(),
+      hadoopConf
+    )
+
+    val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))
+    val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] {
+      var curLine: String = null
+      val mutableRow = new SpecificInternalRow(output.map(_.dataType))
+
+      override def hasNext: Boolean = {
+        try {
+          if (curLine == null) {
+            curLine = reader.readLine()
+            if (curLine == null) {
+              checkFailureAndPropagate(writerThread, 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, 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 class ScriptTransformationWriterThread(
+    iter: Iterator[InternalRow],
+    inputSchema: Seq[DataType],
+    ioschema: ScriptTransformIOSchema,
+    outputStream: OutputStream,
+    proc: Process,
+    stderrBuffer: CircularBuffer,
+    taskContext: TaskContext,
+    conf: Configuration)
+  extends ScriptTransformationWriterThreadBase(
+      iter,
+      inputSchema,
+      outputStream,
+      proc,
+      stderrBuffer,
+      taskContext,
+      conf) {
+
+  setDaemon(true)
+
+  protected val lineDelimiter = ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")
+  protected val fieldDelimiter = ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")
+
+  override def processRows(): Unit = {
+    val len = inputSchema.length
+    iter.foreach { row =>
+      val data = if (len == 0) {
+        ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")
+      } else {
+        val sb = new StringBuilder
+        sb.append(row.get(0, inputSchema(0)))
+        var i = 1
+        while (i < len) {
+          sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"))
+          val columnType = inputSchema(i)
+          val fieldValue = row.get(i, columnType)
+          val fieldStringValue = columnType match {
+            case _: DateType =>
+              val dateFormatter = DateFormatter(DateTimeUtils.defaultTimeZone.toZoneId)
+              dateFormatter.format(fieldValue.asInstanceOf[Int])
+            case _: TimestampType =>
+              DateTimeUtils.timestampToString(
+                TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone.toZoneId),
+                fieldValue.asInstanceOf[SQLTimestamp])

Review comment:
       Two questions I have here;
    - Why do we need the special handling for date/timestamp here? It seems the original one (w/ the null serde case) simply output as strings: https://github.com/apache/spark/blob/6befb2d8bdc5743d0333f4839cf301af165582ce/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala#L289-L306
    - Do you have any plan to support a custom serde in the Spark-native implementation?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/script/ScriptTransformationExec.scala
##########
@@ -0,0 +1,239 @@
+/*
+ * 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 java.sql.Date
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.TaskContext
+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.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.types.{DataType, DateType, TimestampType}
+import org.apache.spark.util.{CircularBuffer, RedirectThread}
+
+/**
+ * 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.
+ */
+case class ScriptTransformationExec(
+    input: Seq[Expression],
+    script: String,
+    output: Seq[Attribute],
+    child: SparkPlan,
+    ioschema: ScriptTransformIOSchema)
+  extends ScriptTransformBase {
+
+  override def producedAttributes: AttributeSet = outputSet -- inputSet
+
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+
+  override def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration)
+  : Iterator[InternalRow] = {
+    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,
+      "Thread-ScriptTransformation-STDERR-Consumer").start()
+
+    val outputProjection = new InterpretedProjection(input, 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 = new ScriptTransformationWriterThread(
+      inputIterator.map(outputProjection),
+      input.map(_.dataType),
+      ioschema,
+      outputStream,
+      proc,
+      stderrBuffer,
+      TaskContext.get(),
+      hadoopConf
+    )
+
+    val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))
+    val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] {
+      var curLine: String = null
+      val mutableRow = new SpecificInternalRow(output.map(_.dataType))
+
+      override def hasNext: Boolean = {
+        try {
+          if (curLine == null) {
+            curLine = reader.readLine()
+            if (curLine == null) {
+              checkFailureAndPropagate(writerThread, 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, 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 class ScriptTransformationWriterThread(
+    iter: Iterator[InternalRow],
+    inputSchema: Seq[DataType],
+    ioschema: ScriptTransformIOSchema,
+    outputStream: OutputStream,
+    proc: Process,
+    stderrBuffer: CircularBuffer,
+    taskContext: TaskContext,
+    conf: Configuration)
+  extends ScriptTransformationWriterThreadBase(
+      iter,
+      inputSchema,
+      outputStream,
+      proc,
+      stderrBuffer,
+      taskContext,
+      conf) {
+
+  setDaemon(true)
+
+  protected val lineDelimiter = ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")
+  protected val fieldDelimiter = ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")
+
+  override def processRows(): Unit = {
+    val len = inputSchema.length
+    iter.foreach { row =>
+      val data = if (len == 0) {
+        ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")
+      } else {
+        val sb = new StringBuilder
+        sb.append(row.get(0, inputSchema(0)))
+        var i = 1
+        while (i < len) {
+          sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"))
+          val columnType = inputSchema(i)
+          val fieldValue = row.get(i, columnType)
+          val fieldStringValue = columnType match {
+            case _: DateType =>
+              val dateFormatter = DateFormatter(DateTimeUtils.defaultTimeZone.toZoneId)
+              dateFormatter.format(fieldValue.asInstanceOf[Int])
+            case _: TimestampType =>
+              DateTimeUtils.timestampToString(
+                TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone.toZoneId),
+                fieldValue.asInstanceOf[SQLTimestamp])

Review comment:
       Two questions I have here;
    - Why do we need the special handling for date/timestamp here? It seems the original one (w/ the null serde case) simply outputs rows as strings though: https://github.com/apache/spark/blob/6befb2d8bdc5743d0333f4839cf301af165582ce/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala#L289-L306
    - Do you have any plan to support a custom serde in the Spark-native implementation?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org