You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by jo...@apache.org on 2014/10/25 03:40:00 UTC

[50/51] [partial] [SQL] Update Hive test harness for Hive 12 and 13

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/.rat-excludes
----------------------------------------------------------------------
diff --git a/.rat-excludes b/.rat-excludes
index b14ad53..ae97456 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -48,6 +48,7 @@ sbt-launch-lib.bash
 plugins.sbt
 work
 .*\.q
+.*\.qv
 golden
 test.out/*
 .*iml

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/dev/run-tests
----------------------------------------------------------------------
diff --git a/dev/run-tests b/dev/run-tests
index 7d06c86..f55497a 100755
--- a/dev/run-tests
+++ b/dev/run-tests
@@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS
   # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled.
   # This must be a single argument, as it is.
   if [ -n "$_RUN_SQL_TESTS" ]; then
-    SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0"
+    SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive"
   fi
   
   if [ -n "$_SQL_TESTS_ONLY" ]; then

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 705937e..ea04473 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -253,7 +253,11 @@ object Hive {
         |import org.apache.spark.sql.hive._
         |import org.apache.spark.sql.hive.test.TestHive._
         |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
-    cleanupCommands in console := "sparkContext.stop()"
+    cleanupCommands in console := "sparkContext.stop()",
+    // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
+    // in order to generate golden files.  This is only required for developers who are adding new
+    // new query tests.
+    fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") }
   )
 
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 7c480de..2b69c02 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -52,6 +52,8 @@ object HiveTypeCoercion {
  */
 trait HiveTypeCoercion {
 
+  import HiveTypeCoercion._
+
   val typeCoercionRules =
     PropagateTypes ::
     ConvertNaNs ::
@@ -340,6 +342,13 @@ trait HiveTypeCoercion {
       // Skip nodes who's children have not been resolved yet.
       case e if !e.childrenResolved => e
 
+      case a @ CreateArray(children) if !a.resolved =>
+        val commonType = a.childTypes.reduce(
+          (a,b) =>
+            findTightestCommonType(a,b).getOrElse(StringType))
+        CreateArray(
+          children.map(c => if (c.dataType == commonType) c else Cast(c, commonType)))
+
       // Promote SUM, SUM DISTINCT and AVERAGE to largest types to prevent overflows.
       case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest.
       case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType))
@@ -356,6 +365,10 @@ trait HiveTypeCoercion {
         Average(Cast(e, LongType))
       case Average(e @ FractionalType()) if e.dataType != DoubleType =>
         Average(Cast(e, DoubleType))
+
+      // Hive lets you do aggregation of timestamps... for some reason
+      case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType))
+      case Average(e @ TimestampType()) => Average(Cast(e, DoubleType))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index dafd745..19421e5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -101,3 +101,28 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio
 
   override def toString = s"$child.$fieldName"
 }
+
+/**
+ * Returns an Array containing the evaluation of all children expressions.
+ */
+case class CreateArray(children: Seq[Expression]) extends Expression {
+  override type EvaluatedType = Any
+
+  lazy val childTypes = children.map(_.dataType).distinct
+
+  override lazy val resolved =
+    childrenResolved && childTypes.size <= 1
+
+  override def dataType: DataType = {
+    assert(resolved, s"Invalid dataType of mixed ArrayType ${childTypes.mkString(",")}")
+    ArrayType(childTypes.headOption.getOrElse(NullType))
+  }
+
+  override def nullable: Boolean = false
+
+  override def eval(input: Row): Any = {
+    children.map(_.eval(input))
+  }
+
+  override def toString = s"Array(${children.mkString(",")})"
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 590dbf3..c4f4ef0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.apache.spark.sql.catalyst.types.DataType
-import org.apache.spark.sql.columnar.InMemoryRelation
 import org.apache.spark.sql.execution.{SparkStrategies, _}
 import org.apache.spark.sql.json._
 import org.apache.spark.sql.parquet.ParquetRelation

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 4638885..15cd62d 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -23,6 +23,7 @@ import java.util.{Locale, TimeZone}
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.sql.SQLConf
+import org.apache.spark.sql.hive.HiveShim
 import org.apache.spark.sql.hive.test.TestHive
 
 /**
@@ -102,6 +103,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "udf5",
     "udf_java_method",
     "create_merge_compressed",
+    "database_location",
+    "database_properties",
 
     // DFS commands
     "symlink_text_input_format",
@@ -135,6 +138,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "stats20",
     "alter_merge_stats",
     "columnstats.*",
+    "annotate_stats.*",
+    "database_drop",
+    "index_serde",
 
 
     // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong.
@@ -211,8 +217,20 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "describe_comment_indent",
 
     // Limit clause without a ordering, which causes failure.
-    "orc_predicate_pushdown"
-  )
+    "orc_predicate_pushdown",
+
+    // Requires precision decimal support:
+    "decimal_1",
+    "udf_pmod",
+    "udf_when",
+    "udf_case",
+    "udf_to_double",
+    "udf_to_float",
+
+    // Needs constant object inspectors
+    "udf_round",
+    "udf7"
+  ) ++ HiveShim.compatibilityBlackList
 
   /**
    * The set of tests that are believed to be working in catalyst. Tests not on whiteList or
@@ -220,23 +238,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
    */
   override def whiteList = Seq(
     "add_part_exist",
-    "dynamic_partition_skip_default",
-    "infer_bucket_sort_dyn_part",
-    "load_dyn_part1",
-    "load_dyn_part2",
-    "load_dyn_part3",
-    "load_dyn_part4",
-    "load_dyn_part5",
-    "load_dyn_part6",
-    "load_dyn_part7",
-    "load_dyn_part8",
-    "load_dyn_part9",
-    "load_dyn_part10",
-    "load_dyn_part11",
-    "load_dyn_part12",
-    "load_dyn_part13",
-    "load_dyn_part14",
-    "load_dyn_part14_win",
     "add_part_multiple",
     "add_partition_no_whitelist",
     "add_partition_with_whitelist",
@@ -256,6 +257,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "alter_varchar2",
     "alter_view_as_select",
     "ambiguous_col",
+    "annotate_stats_join",
+    "annotate_stats_limit",
+    "annotate_stats_part",
+    "annotate_stats_table",
+    "annotate_stats_union",
     "auto_join0",
     "auto_join1",
     "auto_join10",
@@ -299,6 +305,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "auto_sortmerge_join_13",
     "auto_sortmerge_join_14",
     "auto_sortmerge_join_15",
+    "auto_sortmerge_join_16",
     "auto_sortmerge_join_2",
     "auto_sortmerge_join_3",
     "auto_sortmerge_join_4",
@@ -340,7 +347,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "create_skewed_table1",
     "create_struct_table",
     "cross_join",
+    "cross_product_check_1",
+    "cross_product_check_2",
     "ct_case_insensitive",
+    "database_drop",
     "database_location",
     "database_properties",
     "date_2",
@@ -360,8 +370,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "diff_part_input_formats",
     "disable_file_format_check",
     "disallow_incompatible_type_change_off",
+    "distinct_stats",
+    "drop_database_removes_partition_dirs",
     "drop_function",
     "drop_index",
+    "drop_index_removes_partition_dirs",
     "drop_multi_partitions",
     "drop_partitions_filter",
     "drop_partitions_filter2",
@@ -369,23 +382,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "drop_partitions_ignore_protection",
     "drop_table",
     "drop_table2",
+    "drop_table_removes_partition_dirs",
     "drop_view",
+    "dynamic_partition_skip_default",
     "escape_clusterby1",
     "escape_distributeby1",
     "escape_orderby1",
     "escape_sortby1",
+    "explain_rearrange",
     "fetch_aggregation",
+    "fileformat_mix",
     "fileformat_sequencefile",
     "fileformat_text",
     "filter_join_breaktask",
     "filter_join_breaktask2",
     "groupby1",
     "groupby11",
+    "groupby12",
+    "groupby1_limit",
     "groupby1_map",
     "groupby1_map_nomap",
     "groupby1_map_skew",
     "groupby1_noskew",
     "groupby2",
+    "groupby2_limit",
     "groupby2_map",
     "groupby2_map_skew",
     "groupby2_noskew",
@@ -406,6 +426,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "groupby7_map_multi_single_reducer",
     "groupby7_map_skew",
     "groupby7_noskew",
+    "groupby7_noskew_multi_single_reducer",
     "groupby8",
     "groupby8_map",
     "groupby8_map_skew",
@@ -432,6 +453,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "groupby_sort_test_1",
     "having",
     "implicit_cast1",
+    "index_serde",
+    "infer_bucket_sort_dyn_part",
     "innerjoin",
     "inoutdriver",
     "input",
@@ -502,7 +525,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "join17",
     "join18",
     "join19",
-    "join_1to1",
     "join2",
     "join20",
     "join21",
@@ -534,6 +556,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "join7",
     "join8",
     "join9",
+    "join_1to1",
     "join_array",
     "join_casesensitive",
     "join_empty",
@@ -557,7 +580,21 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "literal_double",
     "literal_ints",
     "literal_string",
+    "load_dyn_part1",
+    "load_dyn_part10",
+    "load_dyn_part11",
+    "load_dyn_part12",
+    "load_dyn_part13",
+    "load_dyn_part14",
+    "load_dyn_part14_win",
+    "load_dyn_part2",
+    "load_dyn_part3",
+    "load_dyn_part4",
+    "load_dyn_part5",
+    "load_dyn_part6",
     "load_dyn_part7",
+    "load_dyn_part8",
+    "load_dyn_part9",
     "load_file_with_space_in_the_name",
     "loadpart1",
     "louter_join_ppr",
@@ -578,13 +615,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "merge1",
     "merge2",
     "mergejoins",
-    "multigroupby_singlemr",
+    "multiMapJoin1",
+    "multiMapJoin2",
     "multi_insert_gby",
     "multi_insert_gby3",
     "multi_insert_lateral_view",
     "multi_join_union",
-    "multiMapJoin1",
-    "multiMapJoin2",
+    "multigroupby_singlemr",
     "noalias_subq1",
     "nomore_ambiguous_table_col",
     "nonblock_op_deduplicate",
@@ -607,10 +644,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "outer_join_ppr",
     "parallel",
     "parenthesis_star_by",
-    "partcols1",
     "part_inherit_tbl_props",
     "part_inherit_tbl_props_empty",
     "part_inherit_tbl_props_with_star",
+    "partcols1",
     "partition_date",
     "partition_schema1",
     "partition_serde_format",
@@ -641,7 +678,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "ppd_outer_join5",
     "ppd_random",
     "ppd_repeated_alias",
-    "ppd_transform",
     "ppd_udf_col",
     "ppd_union",
     "ppr_allchildsarenull",
@@ -674,15 +710,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "serde_regex",
     "serde_reported_schema",
     "set_variable_sub",
-    "show_create_table_partitioned",
-    "show_create_table_delimited",
+    "show_columns",
     "show_create_table_alter",
-    "show_create_table_view",
-    "show_create_table_serde",
     "show_create_table_db_table",
+    "show_create_table_delimited",
     "show_create_table_does_not_exist",
     "show_create_table_index",
-    "show_columns",
+    "show_create_table_partitioned",
+    "show_create_table_serde",
+    "show_create_table_view",
     "show_describe_func_quotes",
     "show_functions",
     "show_partitions",
@@ -738,12 +774,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "udaf_covar_pop",
     "udaf_covar_samp",
     "udaf_histogram_numeric",
-    "udf_10_trims",
     "udf2",
     "udf6",
     "udf7",
     "udf8",
     "udf9",
+    "udf_10_trims",
+    "udf_E",
+    "udf_PI",
     "udf_abs",
     "udf_acos",
     "udf_add",
@@ -774,14 +812,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "udf_cos",
     "udf_count",
     "udf_date_add",
-    "udf_datediff",
     "udf_date_sub",
+    "udf_datediff",
     "udf_day",
     "udf_dayofmonth",
     "udf_degrees",
     "udf_div",
     "udf_double",
-    "udf_E",
     "udf_elt",
     "udf_equal",
     "udf_exp",
@@ -826,7 +863,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "udf_nvl",
     "udf_or",
     "udf_parse_url",
-    "udf_PI",
     "udf_pmod",
     "udf_positive",
     "udf_pow",

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 34ed57b..fad4091 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -46,7 +46,6 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs
 import org.apache.spark.sql.execution.QueryExecutionException
 import org.apache.spark.sql.execution.{Command => PhysicalCommand}
 import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
-import org.apache.spark.sql.hive.HiveShim
 
 /**
  * DEPRECATED: Use HiveContext instead.
@@ -230,7 +229,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    * set() or a SET command inside sql() will be set in the SQLConf *as well as*
    * in the HiveConf.
    */
-  @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState])
+  @transient lazy val hiveconf = new HiveConf(classOf[SessionState])
   @transient protected[hive] lazy val sessionState = {
     val ss = new SessionState(hiveconf)
     setConf(hiveconf.getAllProperties)  // Have SQLConf pick up the initial set of HiveConf.

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index deaa1a2..fad7373 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -26,7 +26,6 @@ import org.apache.hadoop.{io => hadoopIo}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types
 import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.hive.HiveShim
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 904bb48..04c48c3 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.hive.HiveShim
 import org.apache.spark.util.Utils
 
 /* Implicit conversions */

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
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 ffcb6b5..54c6197 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
@@ -251,6 +251,8 @@ private[hive] object HiveQl {
         s"""
           |Unsupported language features in query: $sql
           |${dumpTree(getAst(sql))}
+          |$e
+          |${e.getStackTrace.head}
         """.stripMargin)
     }
   }
@@ -329,6 +331,7 @@ private[hive] object HiveQl {
     case Token("TOK_SMALLINT", Nil) => ShortType
     case Token("TOK_BOOLEAN", Nil) => BooleanType
     case Token("TOK_STRING", Nil) => StringType
+    case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType
     case Token("TOK_FLOAT", Nil) => FloatType
     case Token("TOK_DOUBLE", Nil) => DoubleType
     case Token("TOK_DATE", Nil) => DateType
@@ -854,9 +857,11 @@ private[hive] object HiveQl {
     HiveParser.Number,
     HiveParser.TinyintLiteral,
     HiveParser.SmallintLiteral,
-    HiveParser.BigintLiteral)
+    HiveParser.BigintLiteral,
+    HiveParser.DecimalLiteral)
 
   /* Case insensitive matches */
+  val ARRAY = "(?i)ARRAY".r
   val COUNT = "(?i)COUNT".r
   val AVG = "(?i)AVG".r
   val SUM = "(?i)SUM".r
@@ -917,7 +922,9 @@ private[hive] object HiveQl {
     /* Casts */
     case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) =>
       Cast(nodeToExpr(arg), StringType)
-    case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) =>
+    case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), StringType)
+    case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) =>
       Cast(nodeToExpr(arg), StringType)
     case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) =>
       Cast(nodeToExpr(arg), IntegerType)
@@ -1009,6 +1016,8 @@ private[hive] object HiveQl {
       GetItem(nodeToExpr(child), nodeToExpr(ordinal))
 
     /* Other functions */
+    case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) =>
+      CreateArray(children.map(nodeToExpr))
     case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand
     case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) =>
       Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType))
@@ -1042,10 +1051,10 @@ private[hive] object HiveQl {
         } else if (ast.getText.endsWith("Y")) {
           // Literal tinyint.
           v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType)
-        } else if (ast.getText.endsWith("BD")) {
+        } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) {
           // Literal decimal
-          val strVal = ast.getText.substring(0, ast.getText.length() - 2)
-          BigDecimal(strVal)
+          val strVal = ast.getText.stripSuffix("D").stripSuffix("B")
+          v = Literal(BigDecimal(strVal))
         } else {
           v = Literal(ast.getText.toDouble, DoubleType)
           v = Literal(ast.getText.toLong, LongType)
@@ -1056,7 +1065,7 @@ private[hive] object HiveQl {
       }
 
       if (v == null) {
-        sys.error(s"Failed to parse number ${ast.getText}")
+        sys.error(s"Failed to parse number '${ast.getText}'.")
       } else {
         v
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index e45eb57..9ff7ab5 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -34,7 +34,6 @@ import org.apache.spark.SerializableWritable
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.hive.HiveShim
 
 /**
  * A trait for subclasses that handle table scans.

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index c6ff4ea..bb79ad5 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -20,6 +20,9 @@ package org.apache.spark.sql.hive.test
 import java.io.File
 import java.util.{Set => JavaSet}
 
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.session.SessionState
+
 import scala.collection.mutable
 import scala.language.implicitConversions
 
@@ -119,7 +122,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
     if (cmd.toUpperCase contains "LOAD DATA") {
       val testDataLocation =
         hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath)
-      cmd.replaceAll("\\.\\.", testDataLocation)
+      cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/")
     } else {
       cmd
     }
@@ -417,6 +420,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
         FunctionRegistry.unregisterTemporaryUDF(udfName)
       }
 
+      // Some tests corrupt this value on purpose, which breaks the RESET call below.
+      hiveconf.set("fs.default.name", new File(".").toURI.toString)
       // It is important that we RESET first as broken hooks that might have been set could break
       // other sql exec here.
       runSqlHive("RESET")

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
index a201d23..1817c78 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
@@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.api.java
 
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD}
-import org.apache.spark.sql.SQLConf
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.hive.{HiveContext, HiveQl}
 
 /**
  * The entry point for executing Spark SQL queries from a Java program.
  */
-class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) {
+class JavaHiveContext(sqlContext: SQLContext) extends JavaSQLContext(sqlContext) {
 
-  override val sqlContext = new HiveContext(sparkContext)
+  def this(sparkContext: JavaSparkContext) = this(new HiveContext(sparkContext))
 
   override def sql(sqlText: String): JavaSchemaRDD = {
     // TODO: Create a framework for registering parsers instead of just hardcoding if statements.

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index fbd3756..5d98834 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -45,7 +45,7 @@ case class DescribeHiveTableCommand(
   lazy val hiveString: Seq[String] = sideEffectResult.map {
     case Row(name: String, dataType: String, comment) =>
       Seq(name, dataType,
-        Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue))
+        Option(comment.asInstanceOf[String]).getOrElse(""))
         .map(s => String.format(s"%-20s", s))
         .mkString("\t")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/conf/hive-site.xml b/sql/hive/src/test/resources/data/conf/hive-site.xml
index 4e6ff16..7931d6a 100644
--- a/sql/hive/src/test/resources/data/conf/hive-site.xml
+++ b/sql/hive/src/test/resources/data/conf/hive-site.xml
@@ -19,6 +19,12 @@
 
 <configuration>
 
+<property>
+  <name>hive.in.test</name>
+  <value>true</value>
+  <description>Internal marker for test. Used for masking env-dependent values</description>
+</property>
+
 <!-- Hive Configuration can either be stored in this file or in the hadoop configuration files  -->
 <!-- that are implied by Hadoop setup variables.                                                -->
 <!-- Aside from Hadoop setup variables - this file is provided as a convenience so that Hive    -->
@@ -28,7 +34,7 @@
 <!-- Hive Execution Parameters -->
 <property>
   <name>hadoop.tmp.dir</name>
-  <value>${build.dir.hive}/test/hadoop-${user.name}</value>
+  <value>${test.tmp.dir}/hadoop-tmp</value>
   <description>A base for other temporary directories.</description>
 </property>
 
@@ -42,23 +48,28 @@
 
 <property>
   <name>hive.exec.scratchdir</name>
-  <value>${build.dir}/scratchdir</value>
+  <value>${test.tmp.dir}/scratchdir</value>
   <description>Scratch space for Hive jobs</description>
 </property>
 
 <property>
   <name>hive.exec.local.scratchdir</name>
-  <value>${build.dir}/localscratchdir/</value>
+  <value>${test.tmp.dir}/localscratchdir/</value>
   <description>Local scratch space for Hive jobs</description>
 </property>
 
 <property>
   <name>javax.jdo.option.ConnectionURL</name>
-  <!-- note: variable substituion not working here because it's loaded by jdo, not Hive -->
-  <value>jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true</value>
+  <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>
 
 <property>
+  <name>hive.stats.dbconnectionstring</name>
+  <value>jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true</value>
+</property>
+
+
+<property>
   <name>javax.jdo.option.ConnectionDriverName</name>
   <value>org.apache.derby.jdbc.EmbeddedDriver</value>
 </property>
@@ -82,7 +93,7 @@
 
 <property>
   <name>hive.metastore.metadb.dir</name>
-  <value>file://${build.dir}/test/data/metadb/</value>
+  <value>file://${test.tmp.dir}/metadb/</value>
   <description>
   Required by metastore server or if the uris argument below is not supplied
   </description>
@@ -90,32 +101,19 @@
 
 <property>
   <name>test.log.dir</name>
-  <value>${build.dir}/test/logs</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.src.dir</name>
-  <value>file://${build.dir}/src/test</value>
+  <value>${test.tmp.dir}/log/</value>
   <description></description>
 </property>
 
 <property>
   <name>test.data.files</name>
-  <value>${user.dir}/../data/files</value>
-  <description></description>
-</property>
-
-<property>
-  <name>test.query.file1</name>
-  <value>file://${user.dir}/../ql/src/test/org/apache/hadoop/hive/ql/input2.q</value>
-  <value></value>
+  <value>${hive.root}/data/files</value>
   <description></description>
 </property>
 
 <property>
   <name>hive.jar.path</name>
-  <value>${build.dir.hive}/ql/hive-exec-${version}.jar</value>
+  <value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
   <description></description>
 </property>
 
@@ -127,7 +125,7 @@
 
 <property>
   <name>hive.querylog.location</name>
-  <value>${build.dir}/tmp</value>
+  <value>${test.tmp.dir}/tmp</value>
   <description>Location of the structured hive logs</description>
 </property>
 
@@ -144,18 +142,25 @@
 </property>
 
 <property>
-  <name>hive.task.progress</name>
-  <value>false</value>
-  <description>Track progress of a task</description>
-</property>
-
-<property>
   <name>hive.support.concurrency</name>
   <value>true</value>
   <description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
 </property>
 
 <property>
+  <key>hive.unlock.numretries</key>
+  <value>2</value>
+  <description>The number of times you want to retry to do one unlock</description>
+</property>
+
+<property>
+  <key>hive.lock.sleep.between.retries</key>
+  <value>2</value>
+  <description>The sleep time (in seconds) between various retries</description>
+</property>
+
+
+<property>
   <name>fs.pfile.impl</name>
   <value>org.apache.hadoop.fs.ProxyLocalFileSystem</value>
   <description>A proxy for local file system used for cross file system testing</description>
@@ -194,4 +199,21 @@
   <description>The default SerDe hive will use for the rcfile format</description>
 </property>
 
+<property>
+  <name>hive.stats.dbclass</name>
+  <value>jdbc:derby</value>
+  <description>The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported</description>
+</property>
+
+<property>
+  <name>hive.stats.key.prefix.reserve.length</name>
+  <value>0</value>
+</property>
+
+<property>
+  <name>hive.conf.restricted.list</name>
+  <value>dummy.config.value</value>
+  <description>Using dummy config value above because you cannot override config with empty value</description>
+</property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/ProxyAuth.res
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/ProxyAuth.res b/sql/hive/src/test/resources/data/files/ProxyAuth.res
new file mode 100644
index 0000000..96eca8f
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/ProxyAuth.res
@@ -0,0 +1,15 @@
++-----+-------+
+| id  | name  |
++-----+-------+
+| 1   | aaa   |
+| 2   | bbb   |
+| 3   | ccc   |
+| 4   | ddd   |
+| 5   | eee   |
++-----+-------+
++-------+-----+
+| name  | id  |
++-------+-----+
+| aaa   | 1   |
+| bbb   | 2   |
++-------+-----+

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/alltypes.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/alltypes.txt b/sql/hive/src/test/resources/data/files/alltypes.txt
new file mode 100644
index 0000000..358cf40
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/alltypes.txt
@@ -0,0 +1,2 @@
+true|10|100|1000|10000|4.0|20.0|2.2222|1969-12-31 15:59:58.174|1970-01-01 00:00:00|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"}
+true|20|200|2000|20000|8.0|40.0|4.2222|1970-12-31 15:59:58.174|1971-01-01 00:00:00|||k3:v3,k4:v4|200,300|{20, "bar"}

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/alltypes2.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/alltypes2.txt b/sql/hive/src/test/resources/data/files/alltypes2.txt
new file mode 100644
index 0000000..c6a05a1
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/alltypes2.txt
@@ -0,0 +1,2 @@
+true|10|100|1000|10000|4.0|20.0|4.2222|1969-12-31 15:59:58.174|1970-01-01|string|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"}
+false|20|200|2000|20000|8.0|40.0|2.2222|1970-12-31 15:59:58.174|1971-01-01|abcd|world|world|k3:v3,k4:v4|200,300|{20, "bar"}

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/alltypesorc
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/alltypesorc b/sql/hive/src/test/resources/data/files/alltypesorc
new file mode 100644
index 0000000..95c68ca
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/alltypesorc differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/char_varchar_udf.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/char_varchar_udf.txt b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt
new file mode 100644
index 0000000..570078d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt
@@ -0,0 +1 @@
+47.3224	2923.29346

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/datatypes.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/datatypes.txt b/sql/hive/src/test/resources/data/files/datatypes.txt
index 10daa1b..0228a27 100644
--- a/sql/hive/src/test/resources/data/files/datatypes.txt
+++ b/sql/hive/src/test/resources/data/files/datatypes.txt
@@ -1,3 +1,3 @@
-\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N
--1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N
-1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123
+\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N
+-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N\N
+1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123abc123X'01FF'

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/decimal.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/decimal.txt b/sql/hive/src/test/resources/data/files/decimal.txt
new file mode 100644
index 0000000..28800f5
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/decimal.txt
@@ -0,0 +1,18 @@
+55.33
+44.2
+435.33
+324.33
+324.33
+44.2
+55.3
+55.3
+0.0
+
+66.4
+23.22
+-87.2
+
+33.44
+55.3
+435.331
+-0.342
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/dept.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/dept.txt b/sql/hive/src/test/resources/data/files/dept.txt
new file mode 100644
index 0000000..292bee6
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/dept.txt
@@ -0,0 +1,4 @@
+31|sales
+33|engineering
+34|clerical
+35|marketing

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/emp.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/emp.txt b/sql/hive/src/test/resources/data/files/emp.txt
new file mode 100644
index 0000000..a0e76b9
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/emp.txt
@@ -0,0 +1,6 @@
+Rafferty|31
+Jones|33
+Steinberg|33
+Robinson|34
+Smith|34
+John|

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/exported_table/_metadata
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/exported_table/_metadata b/sql/hive/src/test/resources/data/files/exported_table/_metadata
new file mode 100644
index 0000000..81fbf63
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/exported_table/_metadata
@@ -0,0 +1 @@
+{"partitions":[],"table":"{\"1\":{\"str\":\"j1_41\"},\"2\":{\"str\":\"default\"},\"3\":{\"str\":\"johndee\"},\"4\":{\"i32\":1371900915},\"5\":{\"i32\":0},\"6\":{\"i32\":0},\"7\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"8\":{\"lst\":[\"rec\",0]},\"9\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"12\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1"
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/exported_table/data/data
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/exported_table/data/data b/sql/hive/src/test/resources/data/files/exported_table/data/data
new file mode 100644
index 0000000..40a75ac
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/exported_table/data/data
@@ -0,0 +1,2 @@
+johndee,1
+burks,2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt
new file mode 100644
index 0000000..6a39069
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt	
@@ -0,0 +1,3 @@
+12	jason
+13	steven
+15	joe
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/futurama_episodes.avro
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/futurama_episodes.avro b/sql/hive/src/test/resources/data/files/futurama_episodes.avro
new file mode 100644
index 0000000..c08b97b
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/futurama_episodes.avro differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt
new file mode 100644
index 0000000..c242b42
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt
@@ -0,0 +1,8 @@
+name	message	0
+steven	hive	1
+dave	oozie	2
+xifa	phd	3
+chuan	hadoop	4
+shanyu	senior	5
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt
new file mode 100644
index 0000000..d5db38d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt
@@ -0,0 +1,8 @@
+name	message	0
+steven2	hive	11
+dave2	oozie	12
+xifa2	phd	13
+chuan2	hadoop	14
+shanyu2	senior	15
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt
new file mode 100644
index 0000000..f7a763d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt
@@ -0,0 +1,4 @@
+name	message	0
+david3	oozie	22
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt
new file mode 100644
index 0000000..c242b42
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt
@@ -0,0 +1,8 @@
+name	message	0
+steven	hive	1
+dave	oozie	2
+xifa	phd	3
+chuan	hadoop	4
+shanyu	senior	5
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt
new file mode 100644
index 0000000..d5db38d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt
@@ -0,0 +1,8 @@
+name	message	0
+steven2	hive	11
+dave2	oozie	12
+xifa2	phd	13
+chuan2	hadoop	14
+shanyu2	senior	15
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt
new file mode 100644
index 0000000..f7a763d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt
@@ -0,0 +1,4 @@
+name	message	0
+david3	oozie	22
+footer1	footer1	0
+footer2		0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/input.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/input.txt b/sql/hive/src/test/resources/data/files/input.txt
new file mode 100644
index 0000000..caea991
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/input.txt
@@ -0,0 +1,7 @@
+a	b	c	d	e	f	g
+a	b	c	d	e	f	g
+a	b	c	d	e	f	g
+			d	e	f	g
+a	b	c	d			
+a				e	f	g
+a			d			g

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/keystore.jks
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/keystore.jks b/sql/hive/src/test/resources/data/files/keystore.jks
new file mode 100644
index 0000000..469d8a5
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/keystore.jks differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/kv9.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/kv9.txt b/sql/hive/src/test/resources/data/files/kv9.txt
new file mode 100644
index 0000000..b72475f
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv9.txt
@@ -0,0 +1,27 @@
+-4400 4400
+1E+99 0
+1E-99 0
+0 0
+10 10
+23232.23435 2
+2389432.23752 3
+2389432.2375 4
+10.73433 5
+0.333 0
+-0.3 0
+-0.333 0
+1.0 1
+2 2
+3.14 3
+-1.12 -1
+-1.122 -11
+1.12 1
+1.122 1
+124.00 124
+125.2 125
+-1255.49 -1255
+3.14 3
+3.140 4
+0.9999999999999999999999999 1
+-1234567890.1234567890 -1234567890
+1234567890.1234567800 1234567890

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/loc.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/loc.txt b/sql/hive/src/test/resources/data/files/loc.txt
new file mode 100644
index 0000000..69910b7
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/loc.txt
@@ -0,0 +1,8 @@
+OH|31|43201|2001
+IO|32|43202|2001
+CA|35|43809|2001
+FL|33|54342|2001
+UT|35||2001
+CA|35|43809|2001
+|34|40000|
+FL|33|54342|2001

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt
new file mode 100644
index 0000000..41586d6
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt
@@ -0,0 +1 @@
+1|Garçu Kôkaku kidôtai

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/orc_create_people.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/orc_create_people.txt b/sql/hive/src/test/resources/data/files/orc_create_people.txt
index 8845989..ab93c14 100644
--- a/sql/hive/src/test/resources/data/files/orc_create_people.txt
+++ b/sql/hive/src/test/resources/data/files/orc_create_people.txt
@@ -1,100 +1,100 @@
-1CelesteBrowning959-3763 Nec, Av.Ca
-2RisaYangP.O. Box 292, 8229 Porttitor RoadOr
-3VenusSuttonAp #962-8021 Egestas Rd.Ca
-4GretchenHarrisonP.O. Box 636, 8734 Magna AvenueOr
-5LaniIrwinAp #441-5911 Iaculis, AveCa
-6VeraGeorge409-1555 Vel, AveOr
-7JessicaMalone286-9779 Aliquam RoadCa
-8AnnChapmanAp #504-3915 Placerat RoadOr
-9NigelBartlettAp #185-385 Diam StreetCa
-10AzaliaJennings5772 Diam St.Or
-11PrestonCannonAp #527-8769 Nunc AvenueCa
-12AllistairVasquez2562 Odio. St.Or
-13ReedHayes5190 Elit StreetCa
-14ElaineBarronP.O. Box 840, 8860 Sodales Av.Or
-15LydiaHoodP.O. Box 698, 5666 Semper RoadCa
-16VanceMaxwell298-3313 Malesuada RoadOr
-17KeikoDeleonP.O. Box 732, 5921 Massa. Av.Ca
-18DolanKaneAp #906-3606 Ut Rd.Or
-19MerrittPerkinsP.O. Box 228, 7090 Egestas StreetCa
-20CaseySalazar506-5065 Ut St.Or
-21SamsonNoel1370 Ultrices, RoadCa
-22ByronWalkerP.O. Box 386, 8324 Tellus AveOr
-23PiperSingletonAp #500-3561 Primis St.Ca
-24RiaMckinney3080 Dui Rd.Or
-25RahimStanley559-9016 Nascetur StreetCa
-26ChloeSteeleP.O. Box 766, 1628 Elit StreetOr
-27PalomaWardAp #390-3042 Ipsum Rd.Ca
-28RoaryShermanAp #409-6549 Metus St.Or
-29CalvinBuckner6378 Diam AvenueCa
-30CamilleGoodAp #113-8659 Suspendisse St.Or
-31SteelAyala5518 Justo St.Ca
-32JosiahGilbertAp #149-6651 At, Av.Or
-33HamiltonCruz4620 Tellus. AveCa
-34ScarletSantos586-1785 Velit. Av.Or
-35LewisMcintyre629-6419 Ac Rd.Ca
-36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.Or
-37VelmaHaley1377 At Rd.Ca
-38TatumJennings829-7432 Posuere, RoadOr
-39BritanniEaton8811 Morbi StreetCa
-40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.Or
-41KareemAyala2706 Ridiculus StreetCa
-42MaiteRush7592 Neque RoadOr
-43SigneVelasquezAp #868-3039 Eget St.Ca
-44ZoritaCamachoP.O. Box 651, 3340 Quis Av.Or
-45GlennaCurtis953-7965 Enim AveCa
-46QuinCortez4898 Ridiculus St.Or
-47TalonDaltonP.O. Box 408, 7597 Integer Rd.Ca
-48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.Or
-49VernonReyesP.O. Box 971, 7009 Vulputate StreetCa
-50TallulahHeathP.O. Box 865, 3697 Dis AveOr
-51CiaranOlson2721 Et St.Ca
-52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.Or
-53QuinnRiceAp #647-6627 Tristique AvenueCa
-54WyattPickettAp #128-3130 Vel, Rd.Or
-55EmeraldCopeland857-5119 Turpis Rd.Ca
-56JonasQuinnAp #441-7183 Ligula. StreetOr
-57WillaBerg6672 Velit AveCa
-58MalikLee998-9208 In StreetOr
-59CallieMedina1620 Dui. Rd.Ca
-60LukeMasonP.O. Box 143, 2070 Augue Rd.Or
-61ShafiraEstrada8824 Ante StreetCa
-62ElizabethRutledge315-6510 Sit St.Or
-63PandoraLevine357-3596 Nibh. AveCa
-64HilelPrince845-1229 Sociosqu Rd.Or
-65RinahTorresAp #492-9328 At St.Ca
-66YaelHobbsP.O. Box 477, 3896 In StreetOr
-67NevadaNashP.O. Box 251, 1914 Tincidunt RoadCa
-68MarnyHuffP.O. Box 818, 6086 Ultricies St.Or
-69KimberleyMilesAp #893-3685 In RoadCa
-70DuncanFullerAp #197-5216 Iaculis StreetOr
-71YardleyLeblancP.O. Box 938, 1278 Sit AveCa
-72HamishBrewerAp #854-781 Quisque St.Or
-73PetraMoon453-6609 Curabitur StreetCa
-74ReeseEstradaAp #382-3313 Malesuada St.Or
-75GageHiggins7443 Eu StreetCa
-76ZacheryCamachoAp #795-4143 Quam. St.Or
-77KellyGarnerP.O. Box 895, 2843 Cras Rd.Ca
-78HanaeCarr9440 Amet St.Or
-79AnnAlston884-7948 Dictum RoadCa
-80ChancellorCobbP.O. Box 889, 5978 Ac AvenueOr
-81DorothyHarrell6974 Tristique AveCa
-82VaughanLeon1610 Luctus Av.Or
-83WynneJimenez321-9171 Felis. AvenueCa
-84WillaMendoza489-182 Sed Av.Or
-85CamdenGoodwin4579 Ante St.Ca
-86IfeomaFrenchP.O. Box 160, 8769 Integer RoadOr
-87RamonaStrong1666 Ridiculus AvenueCa
-88BrettRamosAp #579-9879 Et, RoadOr
-89UllaGray595-7066 Malesuada RoadCa
-90KevynMccallP.O. Box 968, 1420 Aenean AvenueOr
-91GenevieveWilkins908 Turpis. StreetCa
-92ThaneOneil6766 Lectus St.Or
-93MarikoClineP.O. Box 329, 5375 Ac St.Ca
-94LaelMclean500-7010 Sit St.Or
-95WinifredHopperAp #140-8982 Velit AvenueCa
-96RafaelEnglandP.O. Box 405, 7857 Eget Av.Or
-97DanaCarter814-601 Purus. Av.Ca
-98JulietBattleAp #535-1965 Cursus St.Or
-99WynterVincent626-8492 Mollis AvenueCa
-100WangMitchell4023 Lacinia. AveOr
+1CelesteBrowning959-3763 Nec, Av.100.002011-03-12 15:20:00Ca
+2RisaYangP.O. Box 292, 8229 Porttitor Road200.002011-03-12 15:20:00Or
+3VenusSuttonAp #962-8021 Egestas Rd.300.002011-03-12 15:20:00Ca
+4GretchenHarrisonP.O. Box 636, 8734 Magna Avenue400.002011-03-12 15:20:00Or
+5LaniIrwinAp #441-5911 Iaculis, Ave500.002011-03-12 15:20:00Ca
+6VeraGeorge409-1555 Vel, Ave600.002011-03-12 15:20:00Or
+7JessicaMalone286-9779 Aliquam Road700.002011-03-12 15:20:00Ca
+8AnnChapmanAp #504-3915 Placerat Road800.002011-03-12 15:20:00Or
+9NigelBartlettAp #185-385 Diam Street900.002011-03-12 15:20:00Ca
+10AzaliaJennings5772 Diam St.100.002011-03-12 15:20:00Or
+11PrestonCannonAp #527-8769 Nunc Avenue100.002011-03-12 15:20:00Ca
+12AllistairVasquez2562 Odio. St.100.002011-03-12 15:20:00Or
+13ReedHayes5190 Elit Street100.002011-03-12 15:20:00Ca
+14ElaineBarronP.O. Box 840, 8860 Sodales Av.100.002011-03-12 15:20:00Or
+15LydiaHoodP.O. Box 698, 5666 Semper Road100.002011-03-12 15:20:00Ca
+16VanceMaxwell298-3313 Malesuada Road100.002011-03-12 15:20:00Or
+17KeikoDeleonP.O. Box 732, 5921 Massa. Av.100.002011-03-12 15:20:00Ca
+18DolanKaneAp #906-3606 Ut Rd.100.002011-03-12 15:20:00Or
+19MerrittPerkinsP.O. Box 228, 7090 Egestas Street100.002011-03-12 15:20:00Ca
+20CaseySalazar506-5065 Ut St.200.002011-03-12 15:20:00Or
+21SamsonNoel1370 Ultrices, Road200.002012-03-12 15:20:00Ca
+22ByronWalkerP.O. Box 386, 8324 Tellus Ave200.002012-03-12 15:20:00Or
+23PiperSingletonAp #500-3561 Primis St.200.002012-03-12 15:20:00Ca
+24RiaMckinney3080 Dui Rd.200.002012-03-12 15:20:00Or
+25RahimStanley559-9016 Nascetur Street200.002012-03-12 15:20:00Ca
+26ChloeSteeleP.O. Box 766, 1628 Elit Street200.002012-03-12 15:20:00Or
+27PalomaWardAp #390-3042 Ipsum Rd.200.002012-03-12 15:20:00Ca
+28RoaryShermanAp #409-6549 Metus St.200.002012-03-12 15:20:00Or
+29CalvinBuckner6378 Diam Avenue200.002012-03-12 15:20:00Ca
+30CamilleGoodAp #113-8659 Suspendisse St.300.002012-03-12 15:20:00Or
+31SteelAyala5518 Justo St.300.002012-03-12 15:20:00Ca
+32JosiahGilbertAp #149-6651 At, Av.300.002012-03-12 15:20:00Or
+33HamiltonCruz4620 Tellus. Ave300.002012-03-12 15:20:00Ca
+34ScarletSantos586-1785 Velit. Av.300.002012-03-12 15:20:00Or
+35LewisMcintyre629-6419 Ac Rd.300.002012-03-12 15:20:00Ca
+36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.300.002012-03-12 15:20:00Or
+37VelmaHaley1377 At Rd.300.002012-03-12 15:20:00Ca
+38TatumJennings829-7432 Posuere, Road300.002012-03-12 15:20:00Or
+39BritanniEaton8811 Morbi Street300.002012-03-12 15:20:00Ca
+40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.400.002012-03-12 15:20:00Or
+41KareemAyala2706 Ridiculus Street400.002013-03-12 15:20:00Ca
+42MaiteRush7592 Neque Road400.002013-03-12 15:20:00Or
+43SigneVelasquezAp #868-3039 Eget St.400.002013-03-12 15:20:00Ca
+44ZoritaCamachoP.O. Box 651, 3340 Quis Av.400.002013-03-12 15:20:00Or
+45GlennaCurtis953-7965 Enim Ave400.002013-03-12 15:20:00Ca
+46QuinCortez4898 Ridiculus St.400.002013-03-12 15:20:00Or
+47TalonDaltonP.O. Box 408, 7597 Integer Rd.400.002013-03-12 15:20:00Ca
+48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.400.002013-03-12 15:20:00Or
+49VernonReyesP.O. Box 971, 7009 Vulputate Street400.002013-03-12 15:20:00Ca
+50TallulahHeathP.O. Box 865, 3697 Dis Ave500.002013-03-12 15:20:00Or
+51CiaranOlson2721 Et St.500.002013-03-12 15:20:00Ca
+52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.500.002013-03-12 15:20:00Or
+53QuinnRiceAp #647-6627 Tristique Avenue500.002013-03-12 15:20:00Ca
+54WyattPickettAp #128-3130 Vel, Rd.500.002013-03-12 15:20:00Or
+55EmeraldCopeland857-5119 Turpis Rd.500.002013-03-12 15:20:00Ca
+56JonasQuinnAp #441-7183 Ligula. Street500.002013-03-12 15:20:00Or
+57WillaBerg6672 Velit Ave500.002013-03-12 15:20:00Ca
+58MalikLee998-9208 In Street500.002013-03-12 15:20:00Or
+59CallieMedina1620 Dui. Rd.500.002013-03-12 15:20:00Ca
+60LukeMasonP.O. Box 143, 2070 Augue Rd.600.002013-03-12 15:20:00Or
+61ShafiraEstrada8824 Ante Street600.002014-03-12 15:20:00Ca
+62ElizabethRutledge315-6510 Sit St.600.002014-03-12 15:20:00Or
+63PandoraLevine357-3596 Nibh. Ave600.002014-03-12 15:20:00Ca
+64HilelPrince845-1229 Sociosqu Rd.600.002014-03-12 15:20:00Or
+65RinahTorresAp #492-9328 At St.600.002014-03-12 15:20:00Ca
+66YaelHobbsP.O. Box 477, 3896 In Street600.002014-03-12 15:20:00Or
+67NevadaNashP.O. Box 251, 1914 Tincidunt Road600.002014-03-12 15:20:00Ca
+68MarnyHuffP.O. Box 818, 6086 Ultricies St.600.002014-03-12 15:20:00Or
+69KimberleyMilesAp #893-3685 In Road600.002014-03-12 15:20:00Ca
+70DuncanFullerAp #197-5216 Iaculis Street700.002014-03-12 15:20:00Or
+71YardleyLeblancP.O. Box 938, 1278 Sit Ave700.002014-03-12 15:20:00Ca
+72HamishBrewerAp #854-781 Quisque St.700.002014-03-12 15:20:00Or
+73PetraMoon453-6609 Curabitur Street700.002014-03-12 15:20:00Ca
+74ReeseEstradaAp #382-3313 Malesuada St.700.002014-03-12 15:20:00Or
+75GageHiggins7443 Eu Street700.002014-03-12 15:20:00Ca
+76ZacheryCamachoAp #795-4143 Quam. St.700.002014-03-12 15:20:00Or
+77KellyGarnerP.O. Box 895, 2843 Cras Rd.700.002014-03-12 15:20:00Ca
+78HanaeCarr9440 Amet St.700.002014-03-12 15:20:00Or
+79AnnAlston884-7948 Dictum Road700.002014-03-12 15:20:00Ca
+80ChancellorCobbP.O. Box 889, 5978 Ac Avenue800.002014-03-12 15:20:00Or
+81DorothyHarrell6974 Tristique Ave800.002010-03-12 15:20:00Ca
+82VaughanLeon1610 Luctus Av.800.002010-03-12 15:20:00Or
+83WynneJimenez321-9171 Felis. Avenue800.002010-03-12 15:20:00Ca
+84WillaMendoza489-182 Sed Av.800.002010-03-12 15:20:00Or
+85CamdenGoodwin4579 Ante St.800.002010-03-12 15:20:00Ca
+86IfeomaFrenchP.O. Box 160, 8769 Integer Road800.002010-03-12 15:20:00Or
+87RamonaStrong1666 Ridiculus Avenue800.002010-03-12 15:20:00Ca
+88BrettRamosAp #579-9879 Et, Road800.002010-03-12 15:20:00Or
+89UllaGray595-7066 Malesuada Road800.002010-03-12 15:20:00Ca
+90KevynMccallP.O. Box 968, 1420 Aenean Avenue900.002010-03-12 15:20:00Or
+91GenevieveWilkins908 Turpis. Street900.002010-03-12 15:20:00Ca
+92ThaneOneil6766 Lectus St.900.002010-03-12 15:20:00Or
+93MarikoClineP.O. Box 329, 5375 Ac St.900.002010-03-12 15:20:00Ca
+94LaelMclean500-7010 Sit St.900.002010-03-12 15:20:00Or
+95WinifredHopperAp #140-8982 Velit Avenue900.002010-03-12 15:20:00Ca
+96RafaelEnglandP.O. Box 405, 7857 Eget Av.900.002010-03-12 15:20:00Or
+97DanaCarter814-601 Purus. Av.900.002010-03-12 15:20:00Ca
+98JulietBattleAp #535-1965 Cursus St.900.002010-03-12 15:20:00Or
+99WynterVincent626-8492 Mollis Avenue900.002010-03-12 15:20:00Ca
+100WangMitchell4023 Lacinia. Ave100.002010-03-12 15:20:00Or

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/orc_split_elim.orc
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/orc_split_elim.orc b/sql/hive/src/test/resources/data/files/orc_split_elim.orc
new file mode 100644
index 0000000..cd145d3
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/orc_split_elim.orc differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/parquet_create.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/parquet_create.txt b/sql/hive/src/test/resources/data/files/parquet_create.txt
new file mode 100644
index 0000000..ccd48ee
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/parquet_create.txt
@@ -0,0 +1,3 @@
+1|foo line1|key11:value11,key12:value12,key13:value13|a,b,c|one,two
+2|bar line2|key21:value21,key22:value22,key23:value23|d,e,f|three,four
+3|baz line3|key31:value31,key32:value32,key33:value33|g,h,i|five,six

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/parquet_partitioned.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/parquet_partitioned.txt b/sql/hive/src/test/resources/data/files/parquet_partitioned.txt
new file mode 100644
index 0000000..8f322f3
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/parquet_partitioned.txt
@@ -0,0 +1,3 @@
+1|foo|part1
+2|bar|part2
+3|baz|part2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/parquet_types.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/parquet_types.txt b/sql/hive/src/test/resources/data/files/parquet_types.txt
new file mode 100644
index 0000000..0be390b
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/parquet_types.txt
@@ -0,0 +1,21 @@
+100|1|1|1.0|0.0|abc
+101|2|2|1.1|0.3|def
+102|3|3|1.2|0.6|ghi
+103|1|4|1.3|0.9|jkl
+104|2|5|1.4|1.2|mno
+105|3|1|1.0|1.5|pqr
+106|1|2|1.1|1.8|stu
+107|2|3|1.2|2.1|vwx
+108|3|4|1.3|2.4|yza
+109|1|5|1.4|2.7|bcd
+110|2|1|1.0|3.0|efg
+111|3|2|1.1|3.3|hij
+112|1|3|1.2|3.6|klm
+113|2|4|1.3|3.9|nop
+114|3|5|1.4|4.2|qrs
+115|1|1|1.0|4.5|tuv
+116|2|2|1.1|4.8|wxy
+117|3|3|1.2|5.1|zab
+118|1|4|1.3|5.4|cde
+119|2|5|1.4|5.7|fgh
+120|3|1|1.0|6.0|ijk

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/person age.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/person age.txt b/sql/hive/src/test/resources/data/files/person age.txt
index c902284..f63d241 100644
--- a/sql/hive/src/test/resources/data/files/person age.txt	
+++ b/sql/hive/src/test/resources/data/files/person age.txt	
@@ -1,5 +1,5 @@
-John	23
-Tom	17
-Jim	31
-Boby	9
-Paul	51
\ No newline at end of file
+John23
+Tom17
+Jim31
+Boby9
+Paul51

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/person+age.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/person+age.txt b/sql/hive/src/test/resources/data/files/person+age.txt
new file mode 100644
index 0000000..9d2fa32
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/person+age.txt
@@ -0,0 +1,3 @@
+Sean29
+Tim47
+Pooh21

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/posexplode_data.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/posexplode_data.txt b/sql/hive/src/test/resources/data/files/posexplode_data.txt
new file mode 100644
index 0000000..d04778b
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/posexplode_data.txt
@@ -0,0 +1,4 @@
+John Doe100000.0Mary SmithTodd JonesFederal Taxes.2State Taxes.05Insurance.11 Michigan Ave.ChicagoIL60600
+Mary Smith80000.0Jeremy KingFederal Taxes.2State Taxes. 05Insurance.1100 Ontario St.ChicagoIL60601
+Todd Jones70000.0Federal Taxes.15State Taxes.03Insurance. 1200 Chicago Ave.Oak ParkIL60700
+Jeremy King60000.0Federal Taxes.15State Taxes.03Insurance. 1300 Obscure Dr.ObscuriaIL60100

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/sample.json
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/sample.json b/sql/hive/src/test/resources/data/files/sample.json
new file mode 100644
index 0000000..7b74979
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/sample.json
@@ -0,0 +1 @@
+{"a" : "2" ,"b" : "blah"}

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/symlink1.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/symlink1.txt b/sql/hive/src/test/resources/data/files/symlink1.txt
index dc1a7c5..91d7346 100644
--- a/sql/hive/src/test/resources/data/files/symlink1.txt
+++ b/sql/hive/src/test/resources/data/files/symlink1.txt
@@ -1,2 +1,2 @@
-../data/files/T1.txt
-../data/files/T3.txt
+../../data/files/T1.txt
+../../data/files/T3.txt

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/symlink2.txt
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/symlink2.txt b/sql/hive/src/test/resources/data/files/symlink2.txt
index 8436a30..487b05e 100644
--- a/sql/hive/src/test/resources/data/files/symlink2.txt
+++ b/sql/hive/src/test/resources/data/files/symlink2.txt
@@ -1 +1 @@
-../data/files/T2.txt
+../../data/files/T2.txt

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/files/truststore.jks
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/files/truststore.jks b/sql/hive/src/test/resources/data/files/truststore.jks
new file mode 100644
index 0000000..9c5d703
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/truststore.jks differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/data/scripts/input20_script.py
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/data/scripts/input20_script.py b/sql/hive/src/test/resources/data/scripts/input20_script.py
new file mode 100644
index 0000000..40e3683
--- /dev/null
+++ b/sql/hive/src/test/resources/data/scripts/input20_script.py
@@ -0,0 +1,30 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+import sys
+import re
+line = sys.stdin.readline()
+x = 1
+while line:
+  tem = sys.stdin.readline()
+  if line == tem:
+    x = x + 1
+  else:
+    print str(x).strip()+'\t'+re.sub('\t','_',line.strip())
+    line = tem
+    x = 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb b/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb	
+++ b/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b b/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b	
+++ b/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e b/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e	
+++ b/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 b/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548	
+++ b/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 b/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630	
+++ b/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a b/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a	
+++ b/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a b/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a	
+++ b/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 b/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6	
+++ b/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 b/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6	
+++ b/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e b/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e	
+++ b/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e b/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e	
+++ b/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f b/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f	
+++ b/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 b/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73	
+++ b/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 b/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43	
+++ b/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d b/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d	
+++ b/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 b/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205	
+++ b/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 b/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368	
+++ b/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 b/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43	
+++ b/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a b/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a	
+++ b/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d b/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d	
+++ b/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 b/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8	
+++ b/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f b/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f	
+++ b/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 b/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688	
+++ b/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 b/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9	
+++ b/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 b/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5	
+++ b/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 b/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3	
+++ b/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e b/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e	
+++ b/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 b/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116	
+++ b/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e b/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e	
+++ b/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b b/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b	
+++ b/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b	
@@ -1 +1 @@
-2
\ No newline at end of file
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb b/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb
index 415b19f..cd5ac03 100644
--- a/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb	
+++ b/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0

http://git-wip-us.apache.org/repos/asf/spark/blob/3a845d3c/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f b/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f
index d8263ee..0cfbf08 100644
--- a/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f	
+++ b/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f	
@@ -1 +1 @@
-2
\ No newline at end of file
+2


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