Skip to content

Commit a693722

Browse files
committed
save
1 parent e53744b commit a693722

File tree

3 files changed

+23
-4
lines changed

3 files changed

+23
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution
1919

2020
import java.io.OutputStream
2121
import java.nio.charset.StandardCharsets
22+
import java.time.ZoneId
2223
import java.util.concurrent.TimeUnit
2324

2425
import scala.util.control.NonFatal
@@ -31,8 +32,9 @@ import org.apache.spark.rdd.RDD
3132
import org.apache.spark.sql.catalyst.InternalRow
3233
import org.apache.spark.sql.catalyst.expressions.{AttributeSet, UnsafeProjection}
3334
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
35+
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
3436
import org.apache.spark.sql.internal.SQLConf
35-
import org.apache.spark.sql.types.DataType
37+
import org.apache.spark.sql.types.{DataType, DateType, TimestampType}
3638
import org.apache.spark.util.{CircularBuffer, SerializableConfiguration, Utils}
3739

3840
trait BaseScriptTransformationExec extends UnaryExecNode {
@@ -127,7 +129,19 @@ abstract class BaseScriptTransformationWriterThread extends Thread with Logging
127129
var i = 1
128130
while (i < len) {
129131
sb.append(ioSchema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"))
130-
sb.append(row.get(i, inputSchema(i)))
132+
val columnType = inputSchema(i)
133+
val fieldValue = row.get(i, columnType)
134+
val fieldStringValue = columnType match {
135+
case _: DateType =>
136+
val dateFormatter = DateFormatter(ZoneId.systemDefault())
137+
dateFormatter.format(fieldValue.asInstanceOf[Int])
138+
case _: TimestampType =>
139+
TimestampFormatter.getFractionFormatter(ZoneId.systemDefault())
140+
.format(fieldValue.asInstanceOf[Long])
141+
case _ =>
142+
fieldValue.toString
143+
}
144+
sb.append(fieldStringValue)
131145
i += 1
132146
}
133147
sb.append(ioSchema.inputRowFormatMap("TOK_TABLEROWFORMATLINES"))

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
3535
import org.apache.spark.sql.execution.command._
3636
import org.apache.spark.sql.execution.datasources._
3737
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution}
38+
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
3839
import org.apache.spark.sql.types.StructType
3940

4041
/**
@@ -713,13 +714,16 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
713714
}
714715
(Seq.empty, Option(name), props.toSeq, recordHandler)
715716

716-
case null =>
717+
case null if conf.getConf(CATALOG_IMPLEMENTATION).equals("hive") =>
717718
// Use default (serde) format.
718719
val name = conf.getConfString("hive.script.serde",
719720
"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")
720721
val props = Seq("field.delim" -> "\t")
721722
val recordHandler = Option(conf.getConfString(configKey, defaultConfigValue))
722723
(Nil, Option(name), props, recordHandler)
724+
725+
case null =>
726+
(Nil, None, Seq.empty, None)
723727
}
724728

725729
val (inFormat, inSerdeClass, inSerdeProps, reader) =

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -243,7 +243,8 @@ private[hive] trait HiveStrategies {
243243

244244
object HiveScripts extends Strategy {
245245
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
246-
case ScriptTransformation(input, script, output, child, ioschema) =>
246+
case ScriptTransformation(input, script, output, child, ioschema)
247+
if ioschema.inputSerdeClass.nonEmpty || ioschema.outputSerdeClass.nonEmpty =>
247248
val hiveIoSchema = HiveScriptIOSchema(ioschema)
248249
HiveScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil
249250
case _ => Nil

0 commit comments

Comments
 (0)