From f561c378bc309b34d5bf52a1dcf2a3528f1397e4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 13 Jan 2015 02:08:11 +0800 Subject: [PATCH 01/17] Add support of schema-less script transformation. --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 8 +++++--- .../scala/org/apache/spark/sql/hive/HiveStrategies.scala | 9 ++++++++- .../spark/sql/hive/execution/ScriptTransformation.scala | 8 ++++++-- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 34622b5f5787..20972d9f1d8b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -633,14 +633,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Token(script, Nil) :: Token("TOK_SERDE", serdeClause) :: Token("TOK_RECORDREADER", readerClause) :: - outputClause :: Nil) :: Nil) => + outputClause) :: Nil) => val output = outputClause match { - case Token("TOK_ALIASLIST", aliases) => + case Token("TOK_ALIASLIST", aliases) :: Nil => aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() } - case Token("TOK_TABCOLLIST", attributes) => + case Token("TOK_TABCOLLIST", attributes) :: Nil => attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => AttributeReference(name, nodeToDataType(dataType))() } + case Nil => + Nil } val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index c439b9ebfe10..2239d99d1a0d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -166,7 +166,14 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child) => - ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil + val (newOutput, schemaLess) = + if (output == Nil) { + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) + } else { + (output, false) + } + ScriptTransformation(input, script, newOutput, planLater(child))(hiveContext, schemaLess) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0c8f676e9c5c..8153a9915966 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -40,7 +40,7 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SparkPlan)(@transient sc: HiveContext) + child: SparkPlan)(@transient sc: HiveContext, schemaLess: Boolean) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -61,7 +61,11 @@ case class ScriptTransformation( var curLine = reader.readLine() while (curLine != null) { // TODO: Use SerDe - outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) + if (!schemaLess) { + outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) + } else { + outputLines += new GenericRow(curLine.split("\t", 2).asInstanceOf[Array[Any]]) + } curLine = reader.readLine() } } From ccee49e440c10fa19613d4289fb5f8daeec88bd8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 13 Jan 2015 15:34:25 +0800 Subject: [PATCH 02/17] Add unit test. --- .../spark/sql/hive/execution/HiveQuerySuite.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 700a45edb11d..3b795b48475e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -327,7 +327,19 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") + + test("Schema-Less transform") { + val expected = sql("SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src").collect().head + val res = sql("SELECT TRANSFORM (key) USING 'cat' FROM src").collect().head + + assert(expected(0) === res(0)) + val expected2 = sql("SELECT TRANSFORM (*) USING 'cat' AS (tKey, tValue) FROM src").collect().head + val res2 = sql("SELECT TRANSFORM (*) USING 'cat' FROM src").collect().head + + assert(expected2(0) === res2(0) && expected2(1) === res2(1)) + } + createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") From b1729d97659627cebd1a6344735aa90068961592 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 13 Jan 2015 15:46:25 +0800 Subject: [PATCH 03/17] Fix style. --- .../main/scala/org/apache/spark/sql/hive/HiveStrategies.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 2239d99d1a0d..57c935fe73a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -173,7 +173,8 @@ private[hive] trait HiveStrategies { } else { (output, false) } - ScriptTransformation(input, script, newOutput, planLater(child))(hiveContext, schemaLess) :: Nil + ScriptTransformation(input, script, newOutput, + planLater(child))(hiveContext, schemaLess) :: Nil case _ => Nil } } From 7a48e429e892f92af30b20df8dc6db42797c1f53 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 14 Jan 2015 22:36:01 +0800 Subject: [PATCH 04/17] Add support of custom field delimiter. --- .../plans/logical/ScriptTransformation.scala | 4 +- .../org/apache/spark/sql/hive/HiveQl.scala | 21 ++++++-- .../spark/sql/hive/HiveStrategies.scala | 21 ++++---- .../hive/execution/ScriptTransformation.scala | 53 +++++++++++++------ .../sql/hive/execution/HiveQuerySuite.scala | 13 ++++- 5 files changed, 80 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 4460c86ed902..23d5a5567544 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -30,4 +30,6 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode + child: LogicalPlan, + inputFormat: Seq[(String, String)], + outputFormat: Seq[(String, String)]) extends UnaryNode diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 20972d9f1d8b..7845923bc1b2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -627,11 +627,11 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_SELEXPR", Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", Nil) :: + Token("TOK_SERDE", inputSerdeClause) :: Token("TOK_RECORDWRITER", writerClause) :: // TODO: Need to support other types of (in/out)put Token(script, Nil) :: - Token("TOK_SERDE", serdeClause) :: + Token("TOK_SERDE", outputSerdeClause) :: Token("TOK_RECORDREADER", readerClause) :: outputClause) :: Nil) => @@ -644,6 +644,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Nil => Nil } + + val inputFormat = inputSerdeClause match { + case Token("TOK_SERDEPROPS", props) :: Nil => + props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) } + case Nil => + Nil + } + + val outputFormat = outputSerdeClause match { + case Token("TOK_SERDEPROPS", props) :: Nil => + props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) } + case Nil => + Nil + } + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) Some( @@ -651,7 +666,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C inputExprs.map(nodeToExpr), unescapedScript, output, - withWhere)) + withWhere, inputFormat, outputFormat)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 57c935fe73a5..ab202bcb8056 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -165,16 +165,17 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child) => - val (newOutput, schemaLess) = - if (output == Nil) { - (List(AttributeReference("key", StringType)(), - AttributeReference("value", StringType)()), true) - } else { - (output, false) - } - ScriptTransformation(input, script, newOutput, - planLater(child))(hiveContext, schemaLess) :: Nil + case logical.ScriptTransformation(input, script, output, child, + inputFormat, outputFormat) => + val (newOutput, schemaLess) = + if (output == Nil) { + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) + } else { + (output, false) + } + ScriptTransformation(input, script, newOutput, + planLater(child), inputFormat, outputFormat)(hiveContext, schemaLess) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8153a9915966..f24c36736d98 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -40,11 +40,19 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SparkPlan)(@transient sc: HiveContext, schemaLess: Boolean) + child: SparkPlan, + inputFormat: Seq[(String, String)], + outputFormat: Seq[(String, String)])(@transient sc: HiveContext, schemaLess: Boolean) extends UnaryNode { override def otherCopyArgs = sc :: Nil + val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n")) + + val inputFormatMap = inputFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputFormatMap = outputFormat.toMap.withDefault((k) => defaultFormat(k)) + def execute() = { child.execute().mapPartitions { iter => val cmd = List("/bin/bash", "-c", script) @@ -54,31 +62,42 @@ case class ScriptTransformation( val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) - // TODO: This should be exposed as an iterator instead of reading in all the data at once. - val outputLines = collection.mutable.ArrayBuffer[Row]() - val readerThread = new Thread("Transform OutputReader") { - override def run() { - var curLine = reader.readLine() - while (curLine != null) { - // TODO: Use SerDe - if (!schemaLess) { - outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) - } else { - outputLines += new GenericRow(curLine.split("\t", 2).asInstanceOf[Array[Any]]) - } + val iterator: Iterator[Row] = new Iterator[Row] { + var curLine: String = null + override def hasNext: Boolean = { + if (curLine == null) { curLine = reader.readLine() + curLine != null + } else { + true + } + } + override def next(): Row = { + if (!hasNext) { + throw new NoSuchElementException + } + val prevLine = curLine + curLine = reader.readLine() + // TODO: Use SerDe + if (!schemaLess) { + new GenericRow( + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")).asInstanceOf[Array[Any]]) + } else { + new GenericRow( + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2).asInstanceOf[Array[Any]]) } } } - readerThread.start() + val outputProjection = new InterpretedProjection(input, child.output) iter .map(outputProjection) // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) + .map(_.mkString("", inputFormatMap("TOK_TABLEROWFORMATFIELD"), + inputFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8")) + .foreach(outputStream.write) outputStream.close() - readerThread.join() - outputLines.toIterator + iterator } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 3b795b48475e..50fbc8d08123 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -62,7 +62,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SHOW TABLES") } } - + createQueryTest("! operator", """ |SELECT a FROM ( @@ -339,6 +339,17 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(expected2(0) === res2(0) && expected2(1) === res2(1)) } + + test("transform with custom field delimiter") { + val expected = sql("SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + val res = sql("SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + assert(expected(0) === res(0)) + + val expected2 = sql("SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' AS (tKey, tValue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + val res2 = sql("SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + + assert(expected2(0) === res2(0) && expected2(1) === res2(1)) + } createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") From ab22f7b55988ba324e14969c89d8edfe4d663504 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 14 Jan 2015 22:41:42 +0800 Subject: [PATCH 05/17] Fix style. --- .../spark/sql/hive/execution/ScriptTransformation.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index f24c36736d98..52519397da7b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -81,10 +81,12 @@ case class ScriptTransformation( // TODO: Use SerDe if (!schemaLess) { new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")).asInstanceOf[Array[Any]]) + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")) + .asInstanceOf[Array[Any]]) } else { new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2).asInstanceOf[Array[Any]]) + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .asInstanceOf[Array[Any]]) } } } From 32d3046a228d4bc7f43ed4f20dbd3dee0be42b80 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 18 Jan 2015 23:03:52 +0800 Subject: [PATCH 06/17] Add SerDe support. --- .../plans/logical/ScriptTransformation.scala | 6 +- .../org/apache/spark/sql/hive/HiveQl.scala | 40 +++-- .../spark/sql/hive/HiveStrategies.scala | 7 +- .../hive/execution/ScriptTransformation.scala | 150 ++++++++++++++++-- .../sql/hive/execution/HiveQuerySuite.scala | 13 +- 5 files changed, 185 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 23d5a5567544..802bbbfc3ac8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -32,4 +32,8 @@ case class ScriptTransformation( output: Seq[Attribute], child: LogicalPlan, inputFormat: Seq[(String, String)], - outputFormat: Seq[(String, String)]) extends UnaryNode + outputFormat: Seq[(String, String)], + inputSerdeClass: String, + outputSerdeClass: String, + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)]) extends UnaryNode diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 7845923bc1b2..b5eeb3890c6f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -645,20 +645,38 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Nil } - val inputFormat = inputSerdeClause match { + val (inputFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => - props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) } - case Nil => - Nil + (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, + "", Nil) + case Token("TOK_SERDENAME", Token(serde, Nil) :: Nil) :: Nil => (Nil, serde, Nil) + case Token("TOK_SERDENAME", Token(serde, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", props) :: Nil) :: Nil) :: Nil => + val tableprops = props.map { + case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => + (name, value) + } + (Nil, serde, tableprops) + case Nil => (Nil, "", Nil) } - val outputFormat = outputSerdeClause match { + val (outputFormat, outputSerdeClass, outputSerdeProps) = outputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => - props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) } - case Nil => - Nil + (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, + "", Nil) + case Token("TOK_SERDENAME", Token(serde, Nil) :: Nil) :: Nil => (Nil, serde, Nil) + case Token("TOK_SERDENAME", Token(serde, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", props) :: Nil) :: Nil) :: Nil => + val tableprops = props.map { + case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => + (name, value) + } + (Nil, serde, tableprops) + case Nil => (Nil, "", Nil) } - + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) Some( @@ -666,7 +684,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C inputExprs.map(nodeToExpr), unescapedScript, output, - withWhere, inputFormat, outputFormat)) + withWhere, inputFormat, outputFormat, + inputSerdeClass, outputSerdeClass, + inputSerdeProps, outputSerdeProps)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ab202bcb8056..b85c12613411 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -166,7 +166,8 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child, - inputFormat, outputFormat) => + inputFormat, outputFormat, inputSerdeClass, outputSerdeClass, + inputSerdeProps, outputSerdeProps) => val (newOutput, schemaLess) = if (output == Nil) { (List(AttributeReference("key", StringType)(), @@ -175,7 +176,9 @@ private[hive] trait HiveStrategies { (output, false) } ScriptTransformation(input, script, newOutput, - planLater(child), inputFormat, outputFormat)(hiveContext, schemaLess) :: Nil + planLater(child), inputFormat, outputFormat, + inputSerdeClass, outputSerdeClass, inputSerdeProps, + outputSerdeProps)(hiveContext, schemaLess) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 52519397da7b..87d9deaca894 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,12 +17,25 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader} +import java.io.{BufferedReader, InputStreamReader, ByteArrayInputStream} +import java.io.{DataInputStream, DataOutputStream} +import java.util.Properties + +import org.apache.hadoop.io.{BytesWritable, Text} +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.util.Utils + /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -42,8 +55,12 @@ case class ScriptTransformation( output: Seq[Attribute], child: SparkPlan, inputFormat: Seq[(String, String)], - outputFormat: Seq[(String, String)])(@transient sc: HiveContext, schemaLess: Boolean) - extends UnaryNode { + outputFormat: Seq[(String, String)], + inputSerdeClass: String, + outputSerdeClass: String, + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)])(@transient sc: HiveContext, schemaLess: Boolean) + extends UnaryNode with HiveInspectors { override def otherCopyArgs = sc :: Nil @@ -61,9 +78,43 @@ case class ScriptTransformation( val inputStream = proc.getInputStream val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) + + val outputSerde: Deserializer = if (outputSerdeClass != "") { + val trimed_class = outputSerdeClass.split("'")(1) + Utils.classForName(trimed_class) + .newInstance.asInstanceOf[Deserializer] + } else { + null + } + + if (outputSerde != null) { + val columns = output.map { case aref: AttributeReference => aref.name } + .mkString(",") + val columnTypes = output.map { case aref: AttributeReference => + aref.dataType.toTypeInfo.getTypeName() + }.mkString(",") + + var propsMap = outputSerdeProps.map(kv => { + (kv._1.split("'")(1), kv._2.split("'")(1)) + }).toMap + (serdeConstants.LIST_COLUMNS -> columns) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) + + val properties = new Properties() + properties.putAll(propsMap) + + outputSerde.initialize(null, properties) + } + + val outputSoi = if (outputSerde != null) { + outputSerde.getObjectInspector().asInstanceOf[StructObjectInspector] + } else { + null + } - val iterator: Iterator[Row] = new Iterator[Row] { + val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors { + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) var curLine: String = null + override def hasNext: Boolean = { if (curLine == null) { curLine = reader.readLine() @@ -78,26 +129,91 @@ case class ScriptTransformation( } val prevLine = curLine curLine = reader.readLine() - // TODO: Use SerDe - if (!schemaLess) { - new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")) - .asInstanceOf[Array[Any]]) + + if (outputSerde == null) { + if (!schemaLess) { + new GenericRow( + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")) + .asInstanceOf[Array[Any]]) + } else { + new GenericRow( + prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .asInstanceOf[Array[Any]]) + } } else { - new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2) - .asInstanceOf[Array[Any]]) + val dataInputStream = new DataInputStream( + new ByteArrayInputStream(prevLine.getBytes("utf-8"))) + val writable = new Text() + writable.readFields(dataInputStream) + val raw = outputSerde.deserialize(writable) + val dataList = outputSoi.getStructFieldsDataAsList(raw) + val fieldList = outputSoi.getAllStructFieldRefs() + + var i = 0 + dataList.foreach( element => { + if (element == null) { + mutableRow.setNullAt(i) + } else { + mutableRow(i) = unwrap(element, fieldList(i).getFieldObjectInspector) + } + i += 1 + }) + mutableRow } } } + val inputSerde: Serializer = if (inputSerdeClass != "") { + val trimed_class = inputSerdeClass.split("'")(1) + Utils.classForName(trimed_class) + .newInstance.asInstanceOf[Serializer] + } else { + null + } + + if (inputSerde != null) { + val columns = input.map { case e: NamedExpression => e.name }.mkString(",") + val columnTypes = input.map { case e: NamedExpression => + e.dataType.toTypeInfo.getTypeName() + }.mkString(",") + + var propsMap = inputSerdeProps.map(kv => { + (kv._1.split("'")(1), kv._2.split("'")(1)) + }).toMap + (serdeConstants.LIST_COLUMNS -> columns) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) + + val properties = new Properties() + properties.putAll(propsMap) + inputSerde.initialize(null, properties) + } + + val inputSoi = if (inputSerde != null) { + val fieldNames = input.map { case e: NamedExpression => e.name } + val fieldObjectInspectors = input.map { case e: NamedExpression => + toInspector(e.dataType) + } + ObjectInspectorFactory + .getStandardStructObjectInspector(fieldNames, fieldObjectInspectors) + .asInstanceOf[ObjectInspector] + } else { + null + } + + val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) iter .map(outputProjection) - // TODO: Use SerDe - .map(_.mkString("", inputFormatMap("TOK_TABLEROWFORMATFIELD"), - inputFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8")) - .foreach(outputStream.write) + .foreach { row => + if (inputSerde == null) { + val data = row.mkString("", inputFormatMap("TOK_TABLEROWFORMATFIELD"), + inputFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + + outputStream.write(data) + } else { + inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi).write(dataOutputStream) + outputStream.write(inputFormatMap("TOK_TABLEROWFORMATLINES").getBytes("utf-8")) + } + } outputStream.close() iterator } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 50fbc8d08123..b7fbbcc8b55d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -328,7 +328,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") - test("Schema-Less transform") { + test("schema-less transform") { val expected = sql("SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src").collect().head val res = sql("SELECT TRANSFORM (key) USING 'cat' FROM src").collect().head @@ -351,6 +351,17 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(expected2(0) === res2(0) && expected2(1) === res2(1)) } + test("transform with SerDe") { + val expected = sql("SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) FROM src").collect().head + val res = sql("SELECT TRANSFORM (key, value) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src").collect().head + assert(expected(0) === res(0) && expected(1) === res(1)) + + val expected2 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head + val res2 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head + + assert(expected2(0) === res2(0) && expected2(1) === res2(1)) + } + createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") From be2c3fc81aa990b315715dee3f5f387792cb4617 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 18 Jan 2015 23:21:52 +0800 Subject: [PATCH 07/17] Fix style. --- .../apache/spark/sql/hive/execution/ScriptTransformation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 87d9deaca894..564b27b0b100 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -210,7 +210,8 @@ case class ScriptTransformation( outputStream.write(data) } else { - inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi).write(dataOutputStream) + inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + .write(dataOutputStream) outputStream.write(inputFormatMap("TOK_TABLEROWFORMATLINES").getBytes("utf-8")) } } From 799b5e1a5d18a18b7af5e7db950c40f1a393357e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 19 Jan 2015 03:13:16 +0800 Subject: [PATCH 08/17] Call getSerializedClass instead of using Text. --- .../sql/hive/execution/ScriptTransformation.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 564b27b0b100..b60c3659dd02 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -21,7 +21,6 @@ import java.io.{BufferedReader, InputStreamReader, ByteArrayInputStream} import java.io.{DataInputStream, DataOutputStream} import java.util.Properties -import org.apache.hadoop.io.{BytesWritable, Text} import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.Serializer @@ -79,10 +78,10 @@ case class ScriptTransformation( val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) - val outputSerde: Deserializer = if (outputSerdeClass != "") { + val outputSerde: AbstractSerDe = if (outputSerdeClass != "") { val trimed_class = outputSerdeClass.split("'")(1) Utils.classForName(trimed_class) - .newInstance.asInstanceOf[Deserializer] + .newInstance.asInstanceOf[AbstractSerDe] } else { null } @@ -143,7 +142,7 @@ case class ScriptTransformation( } else { val dataInputStream = new DataInputStream( new ByteArrayInputStream(prevLine.getBytes("utf-8"))) - val writable = new Text() + val writable = outputSerde.getSerializedClass().newInstance writable.readFields(dataInputStream) val raw = outputSerde.deserialize(writable) val dataList = outputSoi.getStructFieldsDataAsList(raw) @@ -163,10 +162,10 @@ case class ScriptTransformation( } } - val inputSerde: Serializer = if (inputSerdeClass != "") { + val inputSerde: AbstractSerDe = if (inputSerdeClass != "") { val trimed_class = inputSerdeClass.split("'")(1) Utils.classForName(trimed_class) - .newInstance.asInstanceOf[Serializer] + .newInstance.asInstanceOf[AbstractSerDe] } else { null } @@ -210,8 +209,8 @@ case class ScriptTransformation( outputStream.write(data) } else { - inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) - .write(dataOutputStream) + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + writable.write(dataOutputStream) outputStream.write(inputFormatMap("TOK_TABLEROWFORMATLINES").getBytes("utf-8")) } } From 7a14f31e73391b97c723bba949a282a3a3c60329 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 19 Jan 2015 17:57:37 +0800 Subject: [PATCH 09/17] Fix bug. --- .../hive/execution/ScriptTransformation.scala | 86 +++++++++++++------ .../sql/hive/execution/HiveQuerySuite.scala | 16 +++- 2 files changed, 70 insertions(+), 32 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index b60c3659dd02..6d9683c2e82d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader, ByteArrayInputStream} -import java.io.{DataInputStream, DataOutputStream} +import java.io.{BufferedReader, InputStreamReader} +import java.io.{DataInputStream, DataOutputStream, EOFException} import java.util.Properties +import java.rmi.server.UID +import org.apache.hadoop.io.Writable +import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.Serializer @@ -111,25 +114,62 @@ case class ScriptTransformation( } val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors { - val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + var cacheRow: Row = null var curLine: String = null + var eof: Boolean = false override def hasNext: Boolean = { - if (curLine == null) { - curLine = reader.readLine() - curLine != null + if (outputSerde == null) { + if (curLine == null) { + curLine = reader.readLine() + curLine != null + } else { + true + } } else { - true + !eof } } + + def deserialize(): Row = { + if (cacheRow != null) return cacheRow + + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + try { + val dataInputStream = new DataInputStream(inputStream) + val writable = outputSerde.getSerializedClass().newInstance + writable.readFields(dataInputStream) + + val raw = outputSerde.deserialize(writable) + val dataList = outputSoi.getStructFieldsDataAsList(raw) + val fieldList = outputSoi.getAllStructFieldRefs() + + var i = 0 + dataList.foreach( element => { + if (element == null) { + mutableRow.setNullAt(i) + } else { + mutableRow(i) = unwrap(element, fieldList(i).getFieldObjectInspector) + } + i += 1 + }) + return mutableRow + } catch { + case e: EOFException => + eof = true + return null + } + } + override def next(): Row = { if (!hasNext) { throw new NoSuchElementException } - val prevLine = curLine - curLine = reader.readLine() if (outputSerde == null) { + val prevLine = curLine + curLine = reader.readLine() + if (!schemaLess) { new GenericRow( prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")) @@ -140,24 +180,12 @@ case class ScriptTransformation( .asInstanceOf[Array[Any]]) } } else { - val dataInputStream = new DataInputStream( - new ByteArrayInputStream(prevLine.getBytes("utf-8"))) - val writable = outputSerde.getSerializedClass().newInstance - writable.readFields(dataInputStream) - val raw = outputSerde.deserialize(writable) - val dataList = outputSoi.getStructFieldsDataAsList(raw) - val fieldList = outputSoi.getAllStructFieldRefs() - - var i = 0 - dataList.foreach( element => { - if (element == null) { - mutableRow.setNullAt(i) - } else { - mutableRow(i) = unwrap(element, fieldList(i).getFieldObjectInspector) - } - i += 1 - }) - mutableRow + val ret = deserialize() + if (!eof) { + cacheRow = null + cacheRow = deserialize() + } + ret } } } @@ -210,8 +238,10 @@ case class ScriptTransformation( outputStream.write(data) } else { val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + if (writable.isInstanceOf[AvroGenericRecordWritable]) { + writable.asInstanceOf[AvroGenericRecordWritable].setRecordReaderID(new UID()) + } writable.write(dataOutputStream) - outputStream.write(inputFormatMap("TOK_TABLEROWFORMATLINES").getBytes("utf-8")) } } outputStream.close() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b7fbbcc8b55d..aaa2f96e4700 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -354,12 +354,20 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("transform with SerDe") { val expected = sql("SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) FROM src").collect().head val res = sql("SELECT TRANSFORM (key, value) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src").collect().head + assert(expected(0) === res(0) && expected(1) === res(1)) - - val expected2 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head - val res2 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head - assert(expected2(0) === res2(0) && expected2(1) === res2(1)) + sql("CREATE TABLE small_src(key INT, value STRING)") + sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") + + val expected2 = sql("SELECT key FROM small_src").collect().head + val res2 = sql("SELECT TRANSFORM (key) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES ('avro.schema.literal'='{\"namespace\": \"testing.hive.avro.serde\",\"name\": \"src\",\"type\": \"record\",\"fields\": [{\"name\":\"key\",\"type\":\"int\"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES ('avro.schema.literal'='{\"namespace\": \"testing.hive.avro.serde\",\"name\": \"src\",\"type\": \"record\",\"fields\": [{\"name\":\"key\",\"type\":\"int\"}]}') FROM small_src").collect().head + assert(expected2(0) === res2(0)) + + val expected3 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head + val res3 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head + + assert(expected3(0) === res3(0) && expected3(1) === res3(1)) } createQueryTest("LIKE", From 9a6dc043a94d6c1999810bf230056c64ee66f623 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 19 Jan 2015 18:31:48 +0800 Subject: [PATCH 10/17] setRecordReaderID is introduced in 0.13.1, use reflection API to call it. --- .../sql/hive/execution/ScriptTransformation.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 6d9683c2e82d..0569a0198a51 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -22,6 +22,8 @@ import java.io.{DataInputStream, DataOutputStream, EOFException} import java.util.Properties import java.rmi.server.UID +import scala.reflect.runtime.universe.{runtimeMirror, newTermName} + import org.apache.hadoop.io.Writable import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable import org.apache.hadoop.hive.serde.serdeConstants @@ -35,7 +37,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveShim} import org.apache.spark.util.Utils @@ -238,8 +240,12 @@ case class ScriptTransformation( outputStream.write(data) } else { val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) - if (writable.isInstanceOf[AvroGenericRecordWritable]) { - writable.asInstanceOf[AvroGenericRecordWritable].setRecordReaderID(new UID()) + if (writable.isInstanceOf[AvroGenericRecordWritable] && HiveShim.version == "0.13.1") { + // setRecordReaderID only in 0.13.1, so use reflection API to call it + val ref = runtimeMirror(getClass.getClassLoader).reflect( + writable.asInstanceOf[AvroGenericRecordWritable]) + val method = ref.symbol.typeSignature.member(newTermName("setRecordReaderID")) + ref.reflectMethod(method.asMethod)(new UID()) } writable.write(dataOutputStream) } From 21727f76f412fa3f71e1a7ec35ba7ca6391f1177 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 23 Jan 2015 18:07:09 +0800 Subject: [PATCH 11/17] Move schema-less output to proper place. Use multilines instead of a long line SQL. --- .../plans/logical/ScriptTransformation.scala | 3 +- .../org/apache/spark/sql/hive/HiveQl.scala | 14 +-- .../spark/sql/hive/HiveStrategies.scala | 13 +-- .../hive/execution/ScriptTransformation.scala | 3 +- .../sql/hive/execution/HiveQuerySuite.scala | 98 +++++++++++++++++-- 5 files changed, 103 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 802bbbfc3ac8..55065a7c925b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -36,4 +36,5 @@ case class ScriptTransformation( inputSerdeClass: String, outputSerdeClass: String, inputSerdeProps: Seq[(String, String)], - outputSerdeProps: Seq[(String, String)]) extends UnaryNode + outputSerdeProps: Seq[(String, String)], + schemaLess: Boolean) extends UnaryNode diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b5eeb3890c6f..e9bdc74e07b1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -635,14 +635,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Token("TOK_RECORDREADER", readerClause) :: outputClause) :: Nil) => - val output = outputClause match { + val (output, schemaLess) = outputClause match { case Token("TOK_ALIASLIST", aliases) :: Nil => - aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() } + (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, + false) case Token("TOK_TABCOLLIST", attributes) :: Nil => - attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => - AttributeReference(name, nodeToDataType(dataType))() } + (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => + AttributeReference(name, nodeToDataType(dataType))() }, false) case Nil => - Nil + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) } val (inputFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { @@ -686,7 +688,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C output, withWhere, inputFormat, outputFormat, inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps)) + inputSerdeProps, outputSerdeProps, schemaLess)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index b85c12613411..2e9f08435dbc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -167,18 +167,11 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child, inputFormat, outputFormat, inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps) => - val (newOutput, schemaLess) = - if (output == Nil) { - (List(AttributeReference("key", StringType)(), - AttributeReference("value", StringType)()), true) - } else { - (output, false) - } - ScriptTransformation(input, script, newOutput, + inputSerdeProps, outputSerdeProps, schemaLess) => + ScriptTransformation(input, script, output, planLater(child), inputFormat, outputFormat, inputSerdeClass, outputSerdeClass, inputSerdeProps, - outputSerdeProps)(hiveContext, schemaLess) :: Nil + outputSerdeProps, schemaLess)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0569a0198a51..0c3f44320f22 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -63,7 +63,8 @@ case class ScriptTransformation( inputSerdeClass: String, outputSerdeClass: String, inputSerdeProps: Seq[(String, String)], - outputSerdeProps: Seq[(String, String)])(@transient sc: HiveContext, schemaLess: Boolean) + outputSerdeProps: Seq[(String, String)], + schemaLess: Boolean)(@transient sc: HiveContext) extends UnaryNode with HiveInspectors { override def otherCopyArgs = sc :: Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index aaa2f96e4700..641ccaeb4402 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -334,26 +334,77 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(expected(0) === res(0)) - val expected2 = sql("SELECT TRANSFORM (*) USING 'cat' AS (tKey, tValue) FROM src").collect().head + val expected2 = sql( + """ + |SELECT TRANSFORM (*) USING 'cat' AS (tKey, tValue) + |FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head val res2 = sql("SELECT TRANSFORM (*) USING 'cat' FROM src").collect().head assert(expected2(0) === res2(0) && expected2(1) === res2(1)) + + val expected3 = sql( + """ + |SELECT TRANSFORM(key + 1, value) USING 'cat' AS (tKey, tValue) + |FROM src ORDER BY tKey, tValue + """.stripMargin.replaceAll("\n", " ")).collect().head + + val res3 = sql( + """ + |SELECT TRANSFORM(key + 1, value) USING 'cat' FROM src + |ORDER BY key, value + """.stripMargin.replaceAll("\n", " ")).collect().head + + assert(expected3(0) === res3(0) && expected3(1) === res3(1)) } test("transform with custom field delimiter") { - val expected = sql("SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head - val res = sql("SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + val delimiter = "'\002'" + + val expected = sql( + s""" + |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head + + val res = sql( + s""" + |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head + assert(expected(0) === res(0)) - val expected2 = sql("SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' AS (tKey, tValue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head - val res2 = sql("SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' FROM src").collect().head + val expected2 = sql( + s""" + |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' AS (tKey, tValue) ROW FORMAT DELIMITED FIELDS TERMINATED + |BY ${delimiter} FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head + + val res2 = sql( + s""" + |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head assert(expected2(0) === res2(0) && expected2(1) === res2(1)) } test("transform with SerDe") { - val expected = sql("SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) FROM src").collect().head - val res = sql("SELECT TRANSFORM (key, value) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src").collect().head + val expected = sql( + """ + |SELECT TRANSFORM (key, value) USING 'cat' + |AS (tKey, tValue) FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head + + val res = sql( + """ + |SELECT TRANSFORM (key, value) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head assert(expected(0) === res(0) && expected(1) === res(1)) @@ -361,11 +412,38 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") val expected2 = sql("SELECT key FROM small_src").collect().head - val res2 = sql("SELECT TRANSFORM (key) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES ('avro.schema.literal'='{\"namespace\": \"testing.hive.avro.serde\",\"name\": \"src\",\"type\": \"record\",\"fields\": [{\"name\":\"key\",\"type\":\"int\"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES ('avro.schema.literal'='{\"namespace\": \"testing.hive.avro.serde\",\"name\": \"src\",\"type\": \"record\",\"fields\": [{\"name\":\"key\",\"type\":\"int\"}]}') FROM small_src").collect().head + val res2 = sql( + """ + |SELECT TRANSFORM (key) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ('avro.schema.literal'='{"namespace": + |"testing.hive.avro.serde","name": "src","type": "record","fields": + |[{"name":"key","type":"int"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES + |('avro.schema.literal'='{"namespace": "testing.hive.avro.serde","name": + |"src","type": "record","fields": [{"name":"key","type":"int"}]}') + |FROM small_src + """.stripMargin.replaceAll("\n", " ")).collect().head + assert(expected2(0) === res2(0)) - val expected3 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head - val res3 = sql("SELECT TRANSFORM (*) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src").collect().head + val expected3 = sql( + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head + + val res3 = sql( + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') FROM src + """.stripMargin.replaceAll("\n", " ")).collect().head assert(expected3(0) === res3(0) && expected3(1) === res3(1)) } From 6000889aab6e8efc67cdad227500e0b198e28928 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 23 Jan 2015 19:10:11 +0800 Subject: [PATCH 12/17] Wrap input and output schema into ScriptInputOutputSchema. --- .../plans/logical/ScriptTransformation.scala | 14 ++++++-- .../org/apache/spark/sql/hive/HiveQl.scala | 13 ++++--- .../spark/sql/hive/HiveStrategies.scala | 8 ++--- .../hive/execution/ScriptTransformation.scala | 35 ++++++++----------- 4 files changed, 36 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 55065a7c925b..35c420d4376e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -25,16 +25,24 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} * @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 ioschema the input and output schema applied in the execution of the script. */ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], child: LogicalPlan, - inputFormat: Seq[(String, String)], - outputFormat: Seq[(String, String)], + ioschema: ScriptInputOutputSchema) extends UnaryNode + +/** + * The wrapper class of input and output schema properties for transforming with script. + * + */ +case class ScriptInputOutputSchema( + inputRowFormat: Seq[(String, String)], + outputRowFormat: Seq[(String, String)], inputSerdeClass: String, outputSerdeClass: String, inputSerdeProps: Seq[(String, String)], outputSerdeProps: Seq[(String, String)], - schemaLess: Boolean) extends UnaryNode + schemaLess: Boolean) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index e9bdc74e07b1..12dc93424b40 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -647,7 +647,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C AttributeReference("value", StringType)()), true) } - val (inputFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { + val (inputRowFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, "", Nil) @@ -663,7 +663,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Nil => (Nil, "", Nil) } - val (outputFormat, outputSerdeClass, outputSerdeProps) = outputSerdeClause match { + val (outputRowFormat, outputSerdeClass, outputSerdeProps) = outputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, "", Nil) @@ -681,14 +681,17 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + val schema = logical.ScriptInputOutputSchema( + inputRowFormat, outputRowFormat, + inputSerdeClass, outputSerdeClass, + inputSerdeProps, outputSerdeProps, schemaLess) + Some( logical.ScriptTransformation( inputExprs.map(nodeToExpr), unescapedScript, output, - withWhere, inputFormat, outputFormat, - inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps, schemaLess)) + withWhere, schema)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 2e9f08435dbc..f65271808462 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -165,13 +165,9 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child, - inputFormat, outputFormat, inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps, schemaLess) => + case logical.ScriptTransformation(input, script, output, child, schema) => ScriptTransformation(input, script, output, - planLater(child), inputFormat, outputFormat, - inputSerdeClass, outputSerdeClass, inputSerdeProps, - outputSerdeProps, schemaLess)(hiveContext) :: Nil + planLater(child), schema)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0c3f44320f22..0fc9073d8f8e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveShim} import org.apache.spark.util.Utils @@ -58,13 +59,7 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: SparkPlan, - inputFormat: Seq[(String, String)], - outputFormat: Seq[(String, String)], - inputSerdeClass: String, - outputSerdeClass: String, - inputSerdeProps: Seq[(String, String)], - outputSerdeProps: Seq[(String, String)], - schemaLess: Boolean)(@transient sc: HiveContext) + ioschema: ScriptInputOutputSchema)(@transient sc: HiveContext) extends UnaryNode with HiveInspectors { override def otherCopyArgs = sc :: Nil @@ -72,8 +67,8 @@ case class ScriptTransformation( val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), ("TOK_TABLEROWFORMATLINES", "\n")) - val inputFormatMap = inputFormat.toMap.withDefault((k) => defaultFormat(k)) - val outputFormatMap = outputFormat.toMap.withDefault((k) => defaultFormat(k)) + val inputRowFormatMap = ioschema.inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = ioschema.outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) def execute() = { child.execute().mapPartitions { iter => @@ -84,8 +79,8 @@ case class ScriptTransformation( val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) - val outputSerde: AbstractSerDe = if (outputSerdeClass != "") { - val trimed_class = outputSerdeClass.split("'")(1) + val outputSerde: AbstractSerDe = if (ioschema.outputSerdeClass != "") { + val trimed_class = ioschema.outputSerdeClass.split("'")(1) Utils.classForName(trimed_class) .newInstance.asInstanceOf[AbstractSerDe] } else { @@ -99,7 +94,7 @@ case class ScriptTransformation( aref.dataType.toTypeInfo.getTypeName() }.mkString(",") - var propsMap = outputSerdeProps.map(kv => { + var propsMap = ioschema.outputSerdeProps.map(kv => { (kv._1.split("'")(1), kv._2.split("'")(1)) }).toMap + (serdeConstants.LIST_COLUMNS -> columns) propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) @@ -173,13 +168,13 @@ case class ScriptTransformation( val prevLine = curLine curLine = reader.readLine() - if (!schemaLess) { + if (!ioschema.schemaLess) { new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD")) + prevLine.split(outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) .asInstanceOf[Array[Any]]) } else { new GenericRow( - prevLine.split(outputFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + prevLine.split(outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) .asInstanceOf[Array[Any]]) } } else { @@ -193,8 +188,8 @@ case class ScriptTransformation( } } - val inputSerde: AbstractSerDe = if (inputSerdeClass != "") { - val trimed_class = inputSerdeClass.split("'")(1) + val inputSerde: AbstractSerDe = if (ioschema.inputSerdeClass != "") { + val trimed_class = ioschema.inputSerdeClass.split("'")(1) Utils.classForName(trimed_class) .newInstance.asInstanceOf[AbstractSerDe] } else { @@ -207,7 +202,7 @@ case class ScriptTransformation( e.dataType.toTypeInfo.getTypeName() }.mkString(",") - var propsMap = inputSerdeProps.map(kv => { + var propsMap = ioschema.inputSerdeProps.map(kv => { (kv._1.split("'")(1), kv._2.split("'")(1)) }).toMap + (serdeConstants.LIST_COLUMNS -> columns) propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) @@ -235,8 +230,8 @@ case class ScriptTransformation( .map(outputProjection) .foreach { row => if (inputSerde == null) { - val data = row.mkString("", inputFormatMap("TOK_TABLEROWFORMATFIELD"), - inputFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + val data = row.mkString("", inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") outputStream.write(data) } else { From ccb71e30b8a65fcea5d0d57865bdf5928ef9a534 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 28 Jan 2015 18:21:06 +0800 Subject: [PATCH 13/17] Refactor codes for comments. --- .../plans/logical/ScriptTransformation.scala | 12 +- .../org/apache/spark/sql/hive/HiveQl.scala | 13 +- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../hive/execution/ScriptTransformation.scala | 201 ++++++++++-------- .../org/apache/spark/sql/hive/Shim12.scala | 8 +- .../org/apache/spark/sql/hive/Shim13.scala | 21 +- 6 files changed, 152 insertions(+), 106 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 35c420d4376e..e4787ac0be7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -32,17 +32,11 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: LogicalPlan, - ioschema: ScriptInputOutputSchema) extends UnaryNode + ioschema: Option[ScriptInputOutputSchema]) extends UnaryNode /** * The wrapper class of input and output schema properties for transforming with script. * */ -case class ScriptInputOutputSchema( - inputRowFormat: Seq[(String, String)], - outputRowFormat: Seq[(String, String)], - inputSerdeClass: String, - outputSerdeClass: String, - inputSerdeProps: Seq[(String, String)], - outputSerdeProps: Seq[(String, String)], - schemaLess: Boolean) +trait ScriptInputOutputSchema + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 82dc1022e4ed..2752dee319a9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.ExplainCommand -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, + HiveScriptIOSchema} import org.apache.spark.sql.types._ /* Implicit conversions */ @@ -646,7 +647,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (List(AttributeReference("key", StringType)(), AttributeReference("value", StringType)()), true) } - + val (inputRowFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, @@ -662,7 +663,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (Nil, serde, tableprops) case Nil => (Nil, "", Nil) } - + val (outputRowFormat, outputSerdeClass, outputSerdeProps) = outputSerdeClause match { case Token("TOK_SERDEPROPS", props) :: Nil => (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, @@ -678,13 +679,13 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (Nil, serde, tableprops) case Nil => (Nil, "", Nil) } - + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) - val schema = logical.ScriptInputOutputSchema( + val schema = Some(HiveScriptIOSchema( inputRowFormat, outputRowFormat, inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps, schemaLess) + inputSerdeProps, outputSerdeProps, schemaLess)) Some( logical.ScriptTransformation( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ea1e68a1c816..18e0f4c36fd6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -169,7 +169,8 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child, schema) => ScriptTransformation(input, script, output, - planLater(child), schema)(hiveContext) :: Nil + planLater(child), schema.map{ case s: HiveScriptIOSchema => s }.get + )(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0fc9073d8f8e..af16f977a254 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -38,7 +38,9 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveShim} +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveShim, ShimWritable} +import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.util.Utils @@ -59,17 +61,11 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: SparkPlan, - ioschema: ScriptInputOutputSchema)(@transient sc: HiveContext) - extends UnaryNode with HiveInspectors { + ioschema: HiveScriptIOSchema)(@transient sc: HiveContext) + extends UnaryNode { override def otherCopyArgs = sc :: Nil - val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), - ("TOK_TABLEROWFORMATLINES", "\n")) - - val inputRowFormatMap = ioschema.inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - val outputRowFormatMap = ioschema.outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - def execute() = { child.execute().mapPartitions { iter => val cmd = List("/bin/bash", "-c", script) @@ -79,37 +75,7 @@ case class ScriptTransformation( val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) - val outputSerde: AbstractSerDe = if (ioschema.outputSerdeClass != "") { - val trimed_class = ioschema.outputSerdeClass.split("'")(1) - Utils.classForName(trimed_class) - .newInstance.asInstanceOf[AbstractSerDe] - } else { - null - } - - if (outputSerde != null) { - val columns = output.map { case aref: AttributeReference => aref.name } - .mkString(",") - val columnTypes = output.map { case aref: AttributeReference => - aref.dataType.toTypeInfo.getTypeName() - }.mkString(",") - - var propsMap = ioschema.outputSerdeProps.map(kv => { - (kv._1.split("'")(1), kv._2.split("'")(1)) - }).toMap + (serdeConstants.LIST_COLUMNS -> columns) - propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) - - val properties = new Properties() - properties.putAll(propsMap) - - outputSerde.initialize(null, properties) - } - - val outputSoi = if (outputSerde != null) { - outputSerde.getObjectInspector().asInstanceOf[StructObjectInspector] - } else { - null - } + val (outputSerde, outputSoi) = ioschema.initOutputSerDe(output) val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors { var cacheRow: Row = null @@ -170,11 +136,11 @@ case class ScriptTransformation( if (!ioschema.schemaLess) { new GenericRow( - prevLine.split(outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) .asInstanceOf[Array[Any]]) } else { new GenericRow( - prevLine.split(outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) .asInstanceOf[Array[Any]]) } } else { @@ -188,61 +154,21 @@ case class ScriptTransformation( } } - val inputSerde: AbstractSerDe = if (ioschema.inputSerdeClass != "") { - val trimed_class = ioschema.inputSerdeClass.split("'")(1) - Utils.classForName(trimed_class) - .newInstance.asInstanceOf[AbstractSerDe] - } else { - null - } - - if (inputSerde != null) { - val columns = input.map { case e: NamedExpression => e.name }.mkString(",") - val columnTypes = input.map { case e: NamedExpression => - e.dataType.toTypeInfo.getTypeName() - }.mkString(",") - - var propsMap = ioschema.inputSerdeProps.map(kv => { - (kv._1.split("'")(1), kv._2.split("'")(1)) - }).toMap + (serdeConstants.LIST_COLUMNS -> columns) - propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypes) - - val properties = new Properties() - properties.putAll(propsMap) - inputSerde.initialize(null, properties) - } - - val inputSoi = if (inputSerde != null) { - val fieldNames = input.map { case e: NamedExpression => e.name } - val fieldObjectInspectors = input.map { case e: NamedExpression => - toInspector(e.dataType) - } - ObjectInspectorFactory - .getStandardStructObjectInspector(fieldNames, fieldObjectInspectors) - .asInstanceOf[ObjectInspector] - } else { - null - } - + val (inputSerde, inputSoi) = ioschema.initInputSerDe(input) val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) + iter .map(outputProjection) .foreach { row => if (inputSerde == null) { - val data = row.mkString("", inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), - inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") outputStream.write(data) } else { - val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) - if (writable.isInstanceOf[AvroGenericRecordWritable] && HiveShim.version == "0.13.1") { - // setRecordReaderID only in 0.13.1, so use reflection API to call it - val ref = runtimeMirror(getClass.getClassLoader).reflect( - writable.asInstanceOf[AvroGenericRecordWritable]) - val method = ref.symbol.typeSignature.member(newTermName("setRecordReaderID")) - ref.reflectMethod(method.asMethod)(new UID()) - } + val writable = new ShimWritable( + inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi)) writable.write(dataOutputStream) } } @@ -251,3 +177,102 @@ case class ScriptTransformation( } } } + +/** + * The wrapper class of Hive input and output schema properties + * + */ +case class HiveScriptIOSchema ( + inputRowFormat: Seq[(String, String)], + outputRowFormat: Seq[(String, String)], + inputSerdeClass: String, + outputSerdeClass: String, + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)], + schemaLess: Boolean) extends ScriptInputOutputSchema with HiveInspectors { + + val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n")) + + val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + + + def initInputSerDe(input: Seq[Expression]): (AbstractSerDe, ObjectInspector) = { + val (columns, columnTypes) = parseAttrs(input) + val serde = initSerDe(inputSerdeClass, columns, columnTypes, inputSerdeProps) + (serde, initInputSoi(serde, columns, columnTypes)) + } + + def initOutputSerDe(output: Seq[Attribute]): (AbstractSerDe, StructObjectInspector) = { + val (columns, columnTypes) = parseAttrs(output) + val serde = initSerDe(outputSerdeClass, columns, columnTypes, outputSerdeProps) + (serde, initOutputputSoi(serde)) + } + + def parseAttrs(attrs: Seq[Expression]): (Seq[String], Seq[DataType]) = { + + val columns = attrs.map { + case aref: AttributeReference => aref.name + case e: NamedExpression => e.name + case _ => null + } + + val columnTypes = attrs.map { + case aref: AttributeReference => aref.dataType + case e: NamedExpression => e.dataType + case _ => null + } + + (columns, columnTypes) + } + + def initSerDe(serdeClassName: String, columns: Seq[String], + columnTypes: Seq[DataType], serdeProps: Seq[(String, String)]): AbstractSerDe = { + + val serde: AbstractSerDe = if (serdeClassName != "") { + val trimed_class = serdeClassName.split("'")(1) + Utils.classForName(trimed_class) + .newInstance.asInstanceOf[AbstractSerDe] + } else { + null + } + + if (serde != null) { + val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") + + var propsMap = serdeProps.map(kv => { + (kv._1.split("'")(1), kv._2.split("'")(1)) + }).toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(",")) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) + + val properties = new Properties() + properties.putAll(propsMap) + serde.initialize(null, properties) + } + + serde + } + + def initInputSoi(inputSerde: AbstractSerDe, columns: Seq[String], columnTypes: Seq[DataType]) + : ObjectInspector = { + + if (inputSerde != null) { + val fieldObjectInspectors = columnTypes.map(toInspector(_)) + ObjectInspectorFactory + .getStandardStructObjectInspector(columns, fieldObjectInspectors) + .asInstanceOf[ObjectInspector] + } else { + null + } + } + + def initOutputputSoi(outputSerde: AbstractSerDe): StructObjectInspector = { + if (outputSerde != null) { + outputSerde.getObjectInspector().asInstanceOf[StructObjectInspector] + } else { + null + } + } +} + diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index c0b7741bc3e5..66714d975b30 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, Primitive import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat import org.apache.spark.sql.types.{Decimal, DecimalType} @@ -241,8 +241,14 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) } } + + implicit def prepareWritable(shimW: ShimWritable): Writable = { + shimW.writable + } } +case class ShimWritable(writable: Writable) + class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) extends FileSinkDesc(dir, tableInfo, compressed) { } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index c04cda7bf153..360eaa8d2c1f 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties +import java.rmi.server.UID import scala.collection.JavaConversions._ import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.{HiveDecimal} @@ -39,6 +40,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjec import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging @@ -395,8 +397,25 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } + + implicit def prepareWritable(shimW: ShimWritable): Writable = { + shimW.writable match { + case w: AvroGenericRecordWritable => + w.setRecordReaderID(new UID()) + case _ => + } + shimW.writable + } } +/* + * Bug introdiced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that + * is needed to initialize before serialization. + * Fix it through wrapper. + */ +case class ShimWritable(writable: Writable) + + /* * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. * Fix it through wrapper. From a422562088c88a1bb3d3005b7424ac1bddb3e801 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 29 Jan 2015 01:22:58 +0800 Subject: [PATCH 14/17] Use createQueryTest for unit tests and remove unnecessary imports. --- .../hive/execution/ScriptTransformation.scala | 5 - .../sql/hive/execution/HiveQuerySuite.scala | 146 ++++++------------ 2 files changed, 50 insertions(+), 101 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index af16f977a254..fdb096e43cd7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -20,12 +20,7 @@ package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} import java.io.{DataInputStream, DataOutputStream, EOFException} import java.util.Properties -import java.rmi.server.UID -import scala.reflect.runtime.universe.{runtimeMirror, newTermName} - -import org.apache.hadoop.io.Writable -import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.Serializer diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 65170c623a04..69e8cb76c217 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -327,92 +327,48 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") - - test("schema-less transform") { - val expected = sql("SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src").collect().head - val res = sql("SELECT TRANSFORM (key) USING 'cat' FROM src").collect().head - - assert(expected(0) === res(0)) - - val expected2 = sql( - """ - |SELECT TRANSFORM (*) USING 'cat' AS (tKey, tValue) - |FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - val res2 = sql("SELECT TRANSFORM (*) USING 'cat' FROM src").collect().head - - assert(expected2(0) === res2(0) && expected2(1) === res2(1)) - - val expected3 = sql( - """ - |SELECT TRANSFORM(key + 1, value) USING 'cat' AS (tKey, tValue) - |FROM src ORDER BY tKey, tValue - """.stripMargin.replaceAll("\n", " ")).collect().head - - val res3 = sql( - """ - |SELECT TRANSFORM(key + 1, value) USING 'cat' FROM src - |ORDER BY key, value - """.stripMargin.replaceAll("\n", " ")).collect().head - assert(expected3(0) === res3(0) && expected3(1) === res3(1)) - } + createQueryTest("schema-less transform", + """ + |SELECT TRANSFORM (key, value) USING 'cat' FROM src; + |SELECT TRANSFORM (*) USING 'cat' FROM src; + """.stripMargin) - test("transform with custom field delimiter") { - val delimiter = "'\002'" + val delimiter = "'\t'" - val expected = sql( - s""" - |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} - |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head + createQueryTest("transform with custom field delimiter", + s""" + |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) - val res = sql( - s""" - |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} - |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head + createQueryTest("transform with custom field delimiter2", + s""" + |SELECT TRANSFORM (key, value) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) - assert(expected(0) === res(0)) + createQueryTest("transform with custom field delimiter3", + s""" + |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) - val expected2 = sql( - s""" - |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} - |USING 'cat' AS (tKey, tValue) ROW FORMAT DELIMITED FIELDS TERMINATED - |BY ${delimiter} FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - - val res2 = sql( - s""" - |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} - |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - - assert(expected2(0) === res2(0) && expected2(1) === res2(1)) - } - - test("transform with SerDe") { - val expected = sql( - """ - |SELECT TRANSFORM (key, value) USING 'cat' - |AS (tKey, tValue) FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - - val res = sql( - """ - |SELECT TRANSFORM (key, value) ROW FORMAT SERDE - |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' - |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE - |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head + createQueryTest("transform with SerDe", + """ + |SELECT TRANSFORM (key, value) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src; + """.stripMargin.replaceAll("\n", " ")) - assert(expected(0) === res(0) && expected(1) === res(1)) + test("transform with SerDe2") { sql("CREATE TABLE small_src(key INT, value STRING)") sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") - val expected2 = sql("SELECT key FROM small_src").collect().head - val res2 = sql( + val expected = sql("SELECT key FROM small_src").collect().head + val res = sql( """ |SELECT TRANSFORM (key) ROW FORMAT SERDE |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' @@ -425,28 +381,26 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |FROM small_src """.stripMargin.replaceAll("\n", " ")).collect().head - assert(expected2(0) === res2(0)) - - val expected3 = sql( - """ - |SELECT TRANSFORM (*) ROW FORMAT SERDE - |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES - |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) - |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' - |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - - val res3 = sql( - """ - |SELECT TRANSFORM (*) ROW FORMAT SERDE - |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES - |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE - |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES - |('serialization.last.column.takes.rest'='true') FROM src - """.stripMargin.replaceAll("\n", " ")).collect().head - - assert(expected3(0) === res3(0) && expected3(1) === res3(1)) + assert(expected(0) === res(0)) } + + createQueryTest("transform with SerDe3", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with SerDe4", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') FROM src; + """.stripMargin.replaceAll("\n", " ")) createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") From 575f69545796900489103c671839aa86c8bd4bc0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 29 Jan 2015 07:57:08 +0800 Subject: [PATCH 15/17] Add Hive golden answer files for new unit tests. --- ...h SerDe-0-cdc393f3914c879787efe523f692b1e0 | 500 ++++++++++++++++++ ... SerDe3-0-58a8b7eb07a949bc44dccb723222957f | 500 ++++++++++++++++++ ... SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 | 500 ++++++++++++++++++ ...limiter-0-703cca3c02ced422feb11dc13b744484 | 500 ++++++++++++++++++ ...limiter-0-82639dda9ba42df817466dffe2929174 | 500 ++++++++++++++++++ ...imiter2-0-e8713b21483e1efb78ee90b61530479b | 500 ++++++++++++++++++ ...imiter2-0-e8d2b2e60551f69bfb44e555f5cff064 | 500 ++++++++++++++++++ ...imiter3-0-d4f4f471819345e9ce1964e281ea5289 | 500 ++++++++++++++++++ 8 files changed, 4000 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 new file mode 100644 index 000000000000..e34118512c1d --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 new file mode 100644 index 000000000000..e34118512c1d --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 From aa10fbd0462e46782688b69664920dc11f4b1990 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 29 Jan 2015 08:54:59 +0800 Subject: [PATCH 16/17] Add Hive golden answer files again. --- ...ansform-0-d5738de14dd6e29da712ec3318f4118f | 500 ++++++++++++++++++ ...ansform-1-49624ef4e2c3cc2040c06660b926219b | 500 ++++++++++++++++++ 2 files changed, 1000 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f create mode 100644 sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b diff --git a/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b new file mode 100644 index 000000000000..7aae61e5eb82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 From ac2d1fe1093f2ad442ff1bd475cec0a38c096e38 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 2 Feb 2015 21:08:31 +0800 Subject: [PATCH 17/17] Refactor codes for comments. --- .../plans/logical/ScriptTransformation.scala | 7 ++- .../org/apache/spark/sql/hive/HiveQl.scala | 54 ++++++++----------- .../spark/sql/hive/HiveStrategies.scala | 6 +-- .../hive/execution/ScriptTransformation.scala | 9 ++-- .../org/apache/spark/sql/hive/Shim12.scala | 6 +-- .../org/apache/spark/sql/hive/Shim13.scala | 24 ++++----- 6 files changed, 43 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index e4787ac0be7d..cfe2c7a39a17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -32,11 +32,10 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: LogicalPlan, - ioschema: Option[ScriptInputOutputSchema]) extends UnaryNode + ioschema: ScriptInputOutputSchema) extends UnaryNode /** - * The wrapper class of input and output schema properties for transforming with script. - * + * A placeholder for implementation specific input and output properties when passing data + * to a script. For example, in Hive this would specify which SerDes to use. */ trait ScriptInputOutputSchema - diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 981666b3822a..ab305e1f82a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -33,8 +33,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.ExplainCommand -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, - HiveScriptIOSchema} +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ /* Implicit conversions */ @@ -648,44 +647,37 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C AttributeReference("value", StringType)()), true) } - val (inputRowFormat, inputSerdeClass, inputSerdeProps) = inputSerdeClause match { - case Token("TOK_SERDEPROPS", props) :: Nil => - (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, - "", Nil) - case Token("TOK_SERDENAME", Token(serde, Nil) :: Nil) :: Nil => (Nil, serde, Nil) - case Token("TOK_SERDENAME", Token(serde, Nil) :: - Token("TOK_TABLEPROPERTIES", - Token("TOK_TABLEPROPLIST", props) :: Nil) :: Nil) :: Nil => - val tableprops = props.map { - case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => - (name, value) - } - (Nil, serde, tableprops) - case Nil => (Nil, "", Nil) - } + def matchSerDe(clause: Seq[ASTNode]) = clause match { + case Token("TOK_SERDEPROPS", propsClause) :: Nil => + val rowFormat = propsClause.map { + case Token(name, Token(value, Nil) :: Nil) => (name, value) + } + (rowFormat, "", Nil) + + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => + (Nil, serdeClass, Nil) - val (outputRowFormat, outputSerdeClass, outputSerdeProps) = outputSerdeClause match { - case Token("TOK_SERDEPROPS", props) :: Nil => - (props.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) }, - "", Nil) - case Token("TOK_SERDENAME", Token(serde, Nil) :: Nil) :: Nil => (Nil, serde, Nil) - case Token("TOK_SERDENAME", Token(serde, Nil) :: - Token("TOK_TABLEPROPERTIES", - Token("TOK_TABLEPROPLIST", props) :: Nil) :: Nil) :: Nil => - val tableprops = props.map { + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => + val serdeProps = propsClause.map { case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => (name, value) } - (Nil, serde, tableprops) + (Nil, serdeClass, serdeProps) + case Nil => (Nil, "", Nil) } + val (inRowFormat, inSerdeClass, inSerdeProps) = matchSerDe(inputSerdeClause) + val (outRowFormat, outSerdeClass, outSerdeProps) = matchSerDe(outputSerdeClause) + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) - val schema = Some(HiveScriptIOSchema( - inputRowFormat, outputRowFormat, - inputSerdeClass, outputSerdeClass, - inputSerdeProps, outputSerdeProps, schemaLess)) + val schema = HiveScriptIOSchema( + inRowFormat, outRowFormat, + inSerdeClass, outSerdeClass, + inSerdeProps, outSerdeProps, schemaLess) Some( logical.ScriptTransformation( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index f2eb08c9715f..83244ce1e372 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -166,10 +166,8 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child, schema) => - ScriptTransformation(input, script, output, - planLater(child), schema.map{ case s: HiveScriptIOSchema => s }.get - )(hiveContext) :: Nil + case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) => + ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index fdb096e43cd7..c54fbb6e2469 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveShim, ShimWritable} +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.util.Utils @@ -162,9 +162,8 @@ case class ScriptTransformation( outputStream.write(data) } else { - val writable = new ShimWritable( - inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi)) - writable.write(dataOutputStream) + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + prepareWritable(writable).write(dataOutputStream) } } outputStream.close() @@ -175,7 +174,6 @@ case class ScriptTransformation( /** * The wrapper class of Hive input and output schema properties - * */ case class HiveScriptIOSchema ( inputRowFormat: Seq[(String, String)], @@ -270,4 +268,3 @@ case class HiveScriptIOSchema ( } } } - diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 66714d975b30..254919e8f6fd 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -242,13 +242,11 @@ private[hive] object HiveShim { } } - implicit def prepareWritable(shimW: ShimWritable): Writable = { - shimW.writable + def prepareWritable(w: Writable): Writable = { + w } } -case class ShimWritable(writable: Writable) - class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) extends FileSinkDesc(dir, tableInfo, compressed) { } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 360eaa8d2c1f..45ca59ae56a3 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -397,27 +397,23 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } - - implicit def prepareWritable(shimW: ShimWritable): Writable = { - shimW.writable match { + + /* + * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that + * is needed to initialize before serialization. + */ + def prepareWritable(w: Writable): Writable = { + w match { case w: AvroGenericRecordWritable => w.setRecordReaderID(new UID()) case _ => - } - shimW.writable + } + w } } /* - * Bug introdiced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that - * is needed to initialize before serialization. - * Fix it through wrapper. - */ -case class ShimWritable(writable: Writable) - - -/* - * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Bug introduced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. * Fix it through wrapper. */ class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean)